Skip to content

Commit 34359c9

Browse files
authored
GH-3055: Disable column statistics for all columns by configuration (#3056)
1 parent 54335a6 commit 34359c9

File tree

6 files changed

+162
-13
lines changed

6 files changed

+162
-13
lines changed

parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,7 @@ public static WriterVersion fromString(String name) {
111111
private final ValuesWriterFactory valuesWriterFactory;
112112
private final int columnIndexTruncateLength;
113113
private final int statisticsTruncateLength;
114+
private final boolean statisticsEnabled;
114115

115116
// The expected NDV (number of distinct values) for each columns
116117
private final ColumnProperty<Long> bloomFilterNDVs;
@@ -141,6 +142,7 @@ private ParquetProperties(Builder builder) {
141142
this.valuesWriterFactory = builder.valuesWriterFactory;
142143
this.columnIndexTruncateLength = builder.columnIndexTruncateLength;
143144
this.statisticsTruncateLength = builder.statisticsTruncateLength;
145+
this.statisticsEnabled = builder.statisticsEnabled;
144146
this.bloomFilterNDVs = builder.bloomFilterNDVs.build();
145147
this.bloomFilterFPPs = builder.bloomFilterFPPs.build();
146148
this.bloomFilterEnabled = builder.bloomFilterEnabled.build();
@@ -334,7 +336,13 @@ public Map<String, String> getExtraMetaData() {
334336
}
335337

336338
public boolean getStatisticsEnabled(ColumnDescriptor column) {
337-
return statistics.getValue(column);
339+
// First check column-specific setting
340+
Boolean columnSetting = statistics.getValue(column);
341+
if (columnSetting != null) {
342+
return columnSetting;
343+
}
344+
// Fall back to global setting
345+
return statisticsEnabled;
338346
}
339347

340348
@Override
@@ -369,6 +377,7 @@ public static class Builder {
369377
private ValuesWriterFactory valuesWriterFactory = DEFAULT_VALUES_WRITER_FACTORY;
370378
private int columnIndexTruncateLength = DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
371379
private int statisticsTruncateLength = DEFAULT_STATISTICS_TRUNCATE_LENGTH;
380+
private boolean statisticsEnabled = DEFAULT_STATISTICS_ENABLED;
372381
private final ColumnProperty.Builder<Long> bloomFilterNDVs;
373382
private final ColumnProperty.Builder<Double> bloomFilterFPPs;
374383
private int maxBloomFilterBytes = DEFAULT_MAX_BLOOM_FILTER_BYTES;
@@ -679,6 +688,11 @@ public Builder withStatisticsEnabled(String columnPath, boolean enabled) {
679688
return this;
680689
}
681690

691+
public Builder withStatisticsEnabled(boolean enabled) {
692+
this.statisticsEnabled = enabled;
693+
return this;
694+
}
695+
682696
public ParquetProperties build() {
683697
ParquetProperties properties = new ParquetProperties(this);
684698
// we pass a constructed but uninitialized factory to ParquetProperties above as currently

parquet-hadoop/README.md

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -509,3 +509,19 @@ If `true` then an attempt will be made to dynamically load the relevant classes;
509509
if not found then the library will use the classic non-vectored reads: it is safe to enable this option on older releases.
510510
**Default value:** `false`
511511

512+
---
513+
514+
**Property:** `parquet.column.statistics.enabled`
515+
**Description:** Whether to enable column statistics collection.
516+
If `true`, statistics will be collected for all columns unless explicitly disabled for specific columns.
517+
If `false`, statistics will be disabled for all columns regardless of column-specific settings.
518+
It is possible to enable or disable statistics for specific columns by appending `#` followed by the column path.
519+
**Default value:** `true`
520+
**Example:**
521+
```java
522+
// Enable statistics for all columns
523+
conf.set("parquet.column.statistics.enabled", true);
524+
// Disable statistics for 'column.path'
525+
conf.set("parquet.column.statistics.enabled#column.path", false);
526+
// The final configuration will be: Enable statistics for all columns except 'column.path'
527+
```

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,6 +156,7 @@ public static enum JobSummaryLevel {
156156
public static final String BLOOM_FILTER_CANDIDATES_NUMBER = "parquet.bloom.filter.candidates.number";
157157
public static final String PAGE_ROW_COUNT_LIMIT = "parquet.page.row.count.limit";
158158
public static final String PAGE_WRITE_CHECKSUM_ENABLED = "parquet.page.write-checksum.enabled";
159+
public static final String STATISTICS_ENABLED = "parquet.column.statistics.enabled";
159160

160161
public static JobSummaryLevel getJobSummaryLevel(Configuration conf) {
161162
String level = conf.get(JOB_SUMMARY_LEVEL);
@@ -388,6 +389,26 @@ public static boolean getPageWriteChecksumEnabled(Configuration conf) {
388389
return conf.getBoolean(PAGE_WRITE_CHECKSUM_ENABLED, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED);
389390
}
390391

392+
public static void setStatisticsEnabled(JobContext jobContext, boolean enabled) {
393+
getConfiguration(jobContext).setBoolean(STATISTICS_ENABLED, enabled);
394+
}
395+
396+
public static boolean getStatisticsEnabled(Configuration conf) {
397+
return conf.getBoolean(STATISTICS_ENABLED, ParquetProperties.DEFAULT_STATISTICS_ENABLED);
398+
}
399+
400+
public static void setStatisticsEnabled(JobContext jobContext, String columnPath, boolean enabled) {
401+
getConfiguration(jobContext).set(STATISTICS_ENABLED + "#" + columnPath, String.valueOf(enabled));
402+
}
403+
404+
public static boolean getStatisticsEnabled(Configuration conf, String columnPath) {
405+
String columnSpecific = conf.get(STATISTICS_ENABLED + "#" + columnPath);
406+
if (columnSpecific != null) {
407+
return Boolean.parseBoolean(columnSpecific);
408+
}
409+
return conf.getBoolean(STATISTICS_ENABLED, ParquetProperties.DEFAULT_STATISTICS_ENABLED);
410+
}
411+
391412
private WriteSupport<T> writeSupport;
392413
private ParquetOutputCommitter committer;
393414

@@ -463,7 +484,8 @@ public RecordWriter<Void, T> getRecordWriter(Configuration conf, Path file, Comp
463484
.withBloomFilterEnabled(getBloomFilterEnabled(conf))
464485
.withAdaptiveBloomFilterEnabled(getAdaptiveBloomFilterEnabled(conf))
465486
.withPageRowCountLimit(getPageRowCountLimit(conf))
466-
.withPageWriteChecksumEnabled(getPageWriteChecksumEnabled(conf));
487+
.withPageWriteChecksumEnabled(getPageWriteChecksumEnabled(conf))
488+
.withStatisticsEnabled(getStatisticsEnabled(conf));
467489
new ColumnConfigParser()
468490
.withColumnConfig(
469491
ENABLE_DICTIONARY, key -> conf.getBoolean(key, false), propsBuilder::withDictionaryEncoding)
@@ -479,6 +501,10 @@ public RecordWriter<Void, T> getRecordWriter(Configuration conf, Path file, Comp
479501
BLOOM_FILTER_CANDIDATES_NUMBER,
480502
key -> conf.getInt(key, ParquetProperties.DEFAULT_BLOOM_FILTER_CANDIDATES_NUMBER),
481503
propsBuilder::withBloomFilterCandidatesNumber)
504+
.withColumnConfig(
505+
STATISTICS_ENABLED,
506+
key -> conf.getBoolean(key, ParquetProperties.DEFAULT_STATISTICS_ENABLED),
507+
propsBuilder::withStatisticsEnabled)
482508
.parseConfig(conf);
483509

484510
ParquetProperties props = propsBuilder.build();

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -906,6 +906,18 @@ public SELF withStatisticsEnabled(String columnPath, boolean enabled) {
906906
return self();
907907
}
908908

909+
/**
910+
* Sets whether statistics are enabled globally. When disabled, statistics will not be collected
911+
* for any column unless explicitly enabled for specific columns.
912+
*
913+
* @param enabled whether to collect statistics globally
914+
* @return this builder for method chaining
915+
*/
916+
public SELF withStatisticsEnabled(boolean enabled) {
917+
encodingPropsBuilder.withStatisticsEnabled(enabled);
918+
return self();
919+
}
920+
909921
/**
910922
* Build a {@link ParquetWriter} with the accumulated configuration.
911923
*

parquet-hadoop/src/test/java/org/apache/parquet/statistics/DataGenerationContext.java

Lines changed: 44 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,30 @@ public abstract static class WriteContext {
4242
protected final boolean enableValidation;
4343
protected final ParquetProperties.WriterVersion version;
4444
protected final Set<String> disableColumnStatistics;
45+
protected final boolean disableAllStatistics;
46+
47+
public WriteContext(
48+
File path,
49+
MessageType schema,
50+
int blockSize,
51+
int pageSize,
52+
boolean enableDictionary,
53+
boolean enableValidation,
54+
ParquetProperties.WriterVersion version,
55+
Set<String> disableColumnStatistics,
56+
boolean disableAllStatistics)
57+
throws IOException {
58+
this.path = path;
59+
this.fsPath = new Path(path.toString());
60+
this.schema = schema;
61+
this.blockSize = blockSize;
62+
this.pageSize = pageSize;
63+
this.enableDictionary = enableDictionary;
64+
this.enableValidation = enableValidation;
65+
this.version = version;
66+
this.disableColumnStatistics = disableColumnStatistics;
67+
this.disableAllStatistics = disableAllStatistics;
68+
}
4569

4670
public WriteContext(
4771
File path,
@@ -52,7 +76,16 @@ public WriteContext(
5276
boolean enableValidation,
5377
ParquetProperties.WriterVersion version)
5478
throws IOException {
55-
this(path, schema, blockSize, pageSize, enableDictionary, enableValidation, version, ImmutableSet.of());
79+
this(
80+
path,
81+
schema,
82+
blockSize,
83+
pageSize,
84+
enableDictionary,
85+
enableValidation,
86+
version,
87+
ImmutableSet.of(),
88+
false);
5689
}
5790

5891
public WriteContext(
@@ -65,15 +98,16 @@ public WriteContext(
6598
ParquetProperties.WriterVersion version,
6699
Set<String> disableColumnStatistics)
67100
throws IOException {
68-
this.path = path;
69-
this.fsPath = new Path(path.toString());
70-
this.schema = schema;
71-
this.blockSize = blockSize;
72-
this.pageSize = pageSize;
73-
this.enableDictionary = enableDictionary;
74-
this.enableValidation = enableValidation;
75-
this.version = version;
76-
this.disableColumnStatistics = disableColumnStatistics;
101+
this(
102+
path,
103+
schema,
104+
blockSize,
105+
pageSize,
106+
enableDictionary,
107+
enableValidation,
108+
version,
109+
disableColumnStatistics,
110+
false);
77111
}
78112

79113
public abstract void write(ParquetWriter<Group> writer) throws IOException;

parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java

Lines changed: 48 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -317,7 +317,29 @@ public DataContext(
317317
ParquetProperties.WriterVersion version,
318318
Set<String> disableColumnStatistics)
319319
throws IOException {
320-
super(path, buildSchema(seed), blockSize, pageSize, enableDictionary, true, version);
320+
this(seed, path, blockSize, pageSize, enableDictionary, version, disableColumnStatistics, false);
321+
}
322+
323+
public DataContext(
324+
long seed,
325+
File path,
326+
int blockSize,
327+
int pageSize,
328+
boolean enableDictionary,
329+
ParquetProperties.WriterVersion version,
330+
Set<String> disableColumnStatistics,
331+
boolean disableAllStatistics)
332+
throws IOException {
333+
super(
334+
path,
335+
buildSchema(seed),
336+
blockSize,
337+
pageSize,
338+
enableDictionary,
339+
true,
340+
version,
341+
disableColumnStatistics,
342+
disableAllStatistics);
321343

322344
this.random = new Random(seed);
323345
this.recordCount = random.nextInt(MAX_TOTAL_ROWS);
@@ -643,4 +665,29 @@ public void testDisableStatistics() throws IOException {
643665
DataGenerationContext.writeAndTest(test);
644666
}
645667
}
668+
669+
@Test
670+
public void testGlobalStatisticsDisabled() throws IOException {
671+
File file = folder.newFile("test_file_global_stats_disabled.parquet");
672+
file.delete();
673+
674+
LOG.info(String.format("RANDOM SEED: %s", RANDOM_SEED));
675+
Random random = new Random(RANDOM_SEED);
676+
677+
int blockSize = (random.nextInt(54) + 10) * MEGABYTE;
678+
int pageSize = (random.nextInt(10) + 1) * MEGABYTE;
679+
680+
// Create context with global statistics disabled
681+
DataContext context = new DataContext(
682+
random.nextLong(),
683+
file,
684+
blockSize,
685+
pageSize,
686+
true, // enable dictionary
687+
ParquetProperties.WriterVersion.PARQUET_2_0,
688+
ImmutableSet.of(), // no specific column statistics disabled
689+
true); // disable all statistics globally
690+
691+
DataGenerationContext.writeAndTest(context);
692+
}
646693
}

0 commit comments

Comments
 (0)