Skip to content
Merged
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 @@ -95,6 +95,14 @@ static SortedRangeSet of(Range first, Range... rest)
return copyOf(first.getType(), rangeList);
}

static SortedRangeSet of(List<Range> rangeList)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Doc would be good to understand what is expected of the arguments.

{
if (rangeList.isEmpty()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

checkArgument(!rangeList.isEmpty(), "cannot use empty rangeList")

throw new IllegalArgumentException("cannot use empty rangeList");
}
return copyOf(rangeList.get(0).getType(), rangeList);
}

/**
* Provided Ranges are unioned together to form the SortedRangeSet
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.fasterxml.jackson.annotation.JsonTypeInfo;

import java.util.Collection;
import java.util.List;

import static java.util.stream.Collectors.toList;

Expand Down Expand Up @@ -83,6 +84,11 @@ static ValueSet ofRanges(Range first, Range... rest)
return SortedRangeSet.of(first, rest);
}

static ValueSet ofRanges(List<Range> ranges)
{
return SortedRangeSet.of(ranges);
}

static ValueSet copyOfRanges(Type type, Collection<Range> ranges)
{
return SortedRangeSet.copyOf(type, ranges);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,6 @@
import static com.facebook.presto.delta.DeltaErrorCode.DELTA_MISSING_DATA;
import static com.facebook.presto.delta.DeltaErrorCode.DELTA_PARQUET_SCHEMA_MISMATCH;
import static com.facebook.presto.delta.DeltaSessionProperties.getParquetMaxReadBlockSize;
import static com.facebook.presto.delta.DeltaSessionProperties.isFailOnCorruptedParquetStatistics;
import static com.facebook.presto.delta.DeltaSessionProperties.isParquetBatchReaderVerificationEnabled;
import static com.facebook.presto.delta.DeltaSessionProperties.isParquetBatchReadsEnabled;
import static com.facebook.presto.delta.DeltaTypeUtils.convertPartitionValue;
Expand Down Expand Up @@ -160,7 +159,6 @@ public ConnectorPageSource createPageSource(
deltaSplit.getFileSize(),
regularColumnHandles,
deltaTableHandle.toSchemaTableName(),
isFailOnCorruptedParquetStatistics(session),
getParquetMaxReadBlockSize(session),
isParquetBatchReadsEnabled(session),
isParquetBatchReaderVerificationEnabled(session),
Expand Down Expand Up @@ -204,7 +202,6 @@ private static ConnectorPageSource createParquetPageSource(
long fileSize,
List<DeltaColumnHandle> columns,
SchemaTableName tableName,
boolean failOnCorruptedParquetStatistics,
DataSize maxReadBlockSize,
boolean batchReaderEnabled,
boolean verificationEnabled,
Expand Down Expand Up @@ -247,7 +244,7 @@ private static ConnectorPageSource createParquetPageSource(
final ParquetDataSource finalDataSource = dataSource;
ImmutableList.Builder<BlockMetaData> blocks = ImmutableList.builder();
for (BlockMetaData block : footerBlocks.build()) {
if (predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain, failOnCorruptedParquetStatistics)) {
if (predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain)) {
blocks.add(block);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
public final class DeltaSessionProperties
{
private static final String CACHE_ENABLED = "cache_enabled";
private static final String PARQUET_FAIL_WITH_CORRUPTED_STATISTICS = "parquet_fail_with_corrupted_statistics";
private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size";
private static final String PARQUET_BATCH_READ_OPTIMIZATION_ENABLED = "parquet_batch_read_optimization_enabled";
private static final String PARQUET_BATCH_READER_VERIFICATION_ENABLED = "parquet_batch_reader_verification_enabled";
Expand All @@ -50,11 +49,6 @@ public DeltaSessionProperties(
"Enable cache for Delta tables",
cacheConfig.isCachingEnabled(),
false),
booleanProperty(
PARQUET_FAIL_WITH_CORRUPTED_STATISTICS,
"Parquet: Fail when scanning Parquet files with corrupted statistics",
hiveClientConfig.isFailOnCorruptedParquetStatistics(),
false),
dataSizeSessionProperty(
PARQUET_MAX_READ_BLOCK_SIZE,
"Parquet: Maximum size of a block to read",
Expand Down Expand Up @@ -87,11 +81,6 @@ public static boolean isCacheEnabled(ConnectorSession session)
return session.getProperty(CACHE_ENABLED, Boolean.class);
}

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

public static DataSize getParquetMaxReadBlockSize(ConnectorSession session)
{
return session.getProperty(PARQUET_MAX_READ_BLOCK_SIZE, DataSize.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
"hive.max-global-split-iterator-threads",
"hive.max-sort-files-per-bucket",
"hive.bucket-writing",
"hive.parquet.fail-on-corrupted-statistics",
"hive.optimized-reader.enabled"})
public class HiveClientConfig
{
Expand Down Expand Up @@ -101,7 +102,6 @@ public class HiveClientConfig
private DataSize textMaxLineLength = new DataSize(100, MEGABYTE);

private boolean useParquetColumnNames;
private boolean failOnCorruptedParquetStatistics = true;
private DataSize parquetMaxReadBlockSize = new DataSize(16, MEGABYTE);

private boolean assumeCanonicalPartitionKeys;
Expand Down Expand Up @@ -935,19 +935,6 @@ public HiveClientConfig setUseParquetColumnNames(boolean useParquetColumnNames)
return this;
}

public boolean isFailOnCorruptedParquetStatistics()
{
return failOnCorruptedParquetStatistics;
}

@Config("hive.parquet.fail-on-corrupted-statistics")
Comment thread
zhenxiao marked this conversation as resolved.
Outdated
@ConfigDescription("Fail when scanning Parquet files with corrupted statistics")
public HiveClientConfig setFailOnCorruptedParquetStatistics(boolean failOnCorruptedParquetStatistics)
{
this.failOnCorruptedParquetStatistics = failOnCorruptedParquetStatistics;
return this;
}

@NotNull
public DataSize getParquetMaxReadBlockSize()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,6 @@ public final class HiveSessionProperties
public static final String RESPECT_TABLE_FORMAT = "respect_table_format";
private static final String CREATE_EMPTY_BUCKET_FILES = "create_empty_bucket_files";
private static final String PARQUET_USE_COLUMN_NAME = "parquet_use_column_names";
private static final String PARQUET_FAIL_WITH_CORRUPTED_STATISTICS = "parquet_fail_with_corrupted_statistics";
private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size";
private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size";
private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size";
Expand Down Expand Up @@ -327,11 +326,6 @@ public HiveSessionProperties(HiveClientConfig hiveClientConfig, OrcFileWriterCon
"Experimental: Parquet: Access Parquet columns using names from the file",
hiveClientConfig.isUseParquetColumnNames(),
false),
booleanProperty(
PARQUET_FAIL_WITH_CORRUPTED_STATISTICS,
"Parquet: Fail when scanning Parquet files with corrupted statistics",
hiveClientConfig.isFailOnCorruptedParquetStatistics(),
false),
dataSizeSessionProperty(
PARQUET_MAX_READ_BLOCK_SIZE,
"Parquet: Maximum size of a block to read",
Expand Down Expand Up @@ -822,11 +816,6 @@ public static boolean isUseParquetColumnNames(ConnectorSession session)
return session.getProperty(PARQUET_USE_COLUMN_NAME, Boolean.class);
}

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

public static DataSize getParquetMaxReadBlockSize(ConnectorSession session)
{
return session.getProperty(PARQUET_MAX_READ_BLOCK_SIZE, DataSize.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,6 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_MISSING_DATA;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH;
import static com.facebook.presto.hive.HiveSessionProperties.getParquetMaxReadBlockSize;
import static com.facebook.presto.hive.HiveSessionProperties.isFailOnCorruptedParquetStatistics;
import static com.facebook.presto.hive.HiveSessionProperties.isParquetBatchReaderVerificationEnabled;
import static com.facebook.presto.hive.HiveSessionProperties.isParquetBatchReadsEnabled;
import static com.facebook.presto.hive.HiveSessionProperties.isUseParquetColumnNames;
Expand Down Expand Up @@ -179,7 +178,6 @@ public Optional<? extends ConnectorPageSource> createPageSource(
columns,
tableName,
isUseParquetColumnNames(session),
isFailOnCorruptedParquetStatistics(session),
getParquetMaxReadBlockSize(session),
isParquetBatchReadsEnabled(session),
isParquetBatchReaderVerificationEnabled(session),
Expand All @@ -202,7 +200,6 @@ public static ConnectorPageSource createParquetPageSource(
List<HiveColumnHandle> columns,
SchemaTableName tableName,
boolean useParquetColumnNames,
boolean failOnCorruptedParquetStatistics,
DataSize maxReadBlockSize,
boolean batchReaderEnabled,
boolean verificationEnabled,
Expand Down Expand Up @@ -252,7 +249,7 @@ public static ConnectorPageSource createParquetPageSource(
final ParquetDataSource finalDataSource = dataSource;
ImmutableList.Builder<BlockMetaData> blocks = ImmutableList.builder();
for (BlockMetaData block : footerBlocks.build()) {
if (predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain, failOnCorruptedParquetStatistics)) {
if (predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain)) {
blocks.add(block);
hiveFileContext.incrementCounter("parquet.blocksRead", 1);
hiveFileContext.incrementCounter("parquet.rowsRead", block.getRowCount());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,6 @@ public void testDefaults()
.setWriteValidationThreads(16)
.setTextMaxLineLength(new DataSize(100, Unit.MEGABYTE))
.setUseParquetColumnNames(false)
.setFailOnCorruptedParquetStatistics(true)
.setParquetMaxReadBlockSize(new DataSize(16, Unit.MEGABYTE))
.setUseOrcColumnNames(false)
.setAssumeCanonicalPartitionKeys(false)
Expand Down Expand Up @@ -207,7 +206,6 @@ public void testExplicitPropertyMappings()
.put("hive.assume-canonical-partition-keys", "true")
.put("hive.text.max-line-length", "13MB")
.put("hive.parquet.use-column-names", "true")
.put("hive.parquet.fail-on-corrupted-statistics", "false")
.put("hive.parquet.max-read-block-size", "66kB")
.put("hive.orc.use-column-names", "true")
.put("hive.orc.bloom-filters.enabled", "true")
Expand Down Expand Up @@ -325,7 +323,6 @@ public void testExplicitPropertyMappings()
.setS3FileSystemType(S3FileSystemType.EMRFS)
.setTextMaxLineLength(new DataSize(13, Unit.MEGABYTE))
.setUseParquetColumnNames(true)
.setFailOnCorruptedParquetStatistics(false)
.setParquetMaxReadBlockSize(new DataSize(66, Unit.KILOBYTE))
.setUseOrcColumnNames(true)
.setAssumeCanonicalPartitionKeys(true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,6 @@
import static com.facebook.presto.hive.CacheQuota.NO_CACHE_CONSTRAINTS;
import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR;
import static com.facebook.presto.hive.HiveSessionProperties.getParquetMaxReadBlockSize;
import static com.facebook.presto.hive.HiveSessionProperties.isFailOnCorruptedParquetStatistics;
import static com.facebook.presto.hive.HiveSessionProperties.isParquetBatchReaderVerificationEnabled;
import static com.facebook.presto.hive.HiveSessionProperties.isParquetBatchReadsEnabled;
import static com.facebook.presto.hive.HiveSessionProperties.isUseParquetColumnNames;
Expand Down Expand Up @@ -230,7 +229,6 @@ private ConnectorPageSource createDataPageSource(
tableName,
dataColumns,
isUseParquetColumnNames(session),
isFailOnCorruptedParquetStatistics(session),
getParquetMaxReadBlockSize(session),
isParquetBatchReadsEnabled(session),
isParquetBatchReaderVerificationEnabled(session),
Expand Down Expand Up @@ -290,7 +288,6 @@ private static ConnectorPageSource createParquetPageSource(
SchemaTableName tableName,
List<IcebergColumnHandle> regularColumns,
boolean useParquetColumnNames,
boolean failOnCorruptedParquetStatistics,
DataSize maxReadBlockSize,
boolean batchReaderEnabled,
boolean verificationEnabled,
Expand Down Expand Up @@ -338,7 +335,7 @@ private static ConnectorPageSource createParquetPageSource(
for (BlockMetaData block : parquetMetadata.getBlocks()) {
long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
if ((firstDataPage >= start) && (firstDataPage < (start + length)) &&
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, failOnCorruptedParquetStatistics)) {
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) {
blocks.add(block);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
public final class IcebergSessionProperties
{
private static final String COMPRESSION_CODEC = "compression_codec";
private static final String PARQUET_FAIL_WITH_CORRUPTED_STATISTICS = "parquet_fail_with_corrupted_statistics";
private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size";
private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size";
private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size";
Expand Down Expand Up @@ -92,11 +91,6 @@ public IcebergSessionProperties(
false,
value -> HiveCompressionCodec.valueOf(((String) value).toUpperCase()),
HiveCompressionCodec::name),
booleanProperty(
PARQUET_FAIL_WITH_CORRUPTED_STATISTICS,
"Parquet: Fail when scanning Parquet files with corrupted statistics",
hiveClientConfig.isFailOnCorruptedParquetStatistics(),
false),
booleanProperty(
PARQUET_USE_COLUMN_NAMES,
"Experimental: Parquet: Access Parquet columns using names from the file",
Expand Down Expand Up @@ -259,11 +253,6 @@ public static HiveCompressionCodec getCompressionCodec(ConnectorSession session)
return session.getProperty(COMPRESSION_CODEC, HiveCompressionCodec.class);
}

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

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

This file was deleted.

This file was deleted.

This file was deleted.

Loading