This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new ccac04f84 GH-3059: Add configuration to disable size statistics (#3060)
ccac04f84 is described below
commit ccac04f84f971a1eaf390535b23c2cb42c290f9a
Author: Gang Wu <[email protected]>
AuthorDate: Mon Nov 18 09:35:03 2024 +0800
GH-3059: Add configuration to disable size statistics (#3060)
---
.../apache/parquet/column/ParquetProperties.java | 45 +++++++++++++-
.../parquet/column/impl/ColumnValueCollector.java | 9 ++-
.../parquet/column/statistics/SizeStatistics.java | 36 +++++++++++-
parquet-hadoop/README.md | 17 ++++++
.../apache/parquet/hadoop/ParquetOutputFormat.java | 17 ++++++
.../org/apache/parquet/hadoop/ParquetWriter.java | 24 ++++++++
.../apache/parquet/hadoop/TestParquetWriter.java | 68 ++++++++++++++++++++++
7 files changed, 212 insertions(+), 4 deletions(-)
diff --git
a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
index 1fd7ed9bc..9aaef4b3c 100644
---
a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
+++
b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
@@ -64,6 +64,7 @@ public class ParquetProperties {
public static final boolean DEFAULT_ADAPTIVE_BLOOM_FILTER_ENABLED = false;
public static final int DEFAULT_BLOOM_FILTER_CANDIDATES_NUMBER = 5;
public static final boolean DEFAULT_STATISTICS_ENABLED = true;
+ public static final boolean DEFAULT_SIZE_STATISTICS_ENABLED = true;
public static final boolean DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED = true;
@@ -112,6 +113,7 @@ public class ParquetProperties {
private final int columnIndexTruncateLength;
private final int statisticsTruncateLength;
private final boolean statisticsEnabled;
+ private final boolean sizeStatisticsEnabled;
// The expected NDV (number of distinct values) for each columns
private final ColumnProperty<Long> bloomFilterNDVs;
@@ -125,6 +127,7 @@ public class ParquetProperties {
private final ColumnProperty<ByteStreamSplitMode> byteStreamSplitEnabled;
private final Map<String, String> extraMetaData;
private final ColumnProperty<Boolean> statistics;
+ private final ColumnProperty<Boolean> sizeStatistics;
private ParquetProperties(Builder builder) {
this.pageSizeThreshold = builder.pageSize;
@@ -143,6 +146,7 @@ public class ParquetProperties {
this.columnIndexTruncateLength = builder.columnIndexTruncateLength;
this.statisticsTruncateLength = builder.statisticsTruncateLength;
this.statisticsEnabled = builder.statisticsEnabled;
+ this.sizeStatisticsEnabled = builder.sizeStatisticsEnabled;
this.bloomFilterNDVs = builder.bloomFilterNDVs.build();
this.bloomFilterFPPs = builder.bloomFilterFPPs.build();
this.bloomFilterEnabled = builder.bloomFilterEnabled.build();
@@ -154,6 +158,7 @@ public class ParquetProperties {
this.byteStreamSplitEnabled = builder.byteStreamSplitEnabled.build();
this.extraMetaData = builder.extraMetaData;
this.statistics = builder.statistics.build();
+ this.sizeStatistics = builder.sizeStatistics.build();
}
public static Builder builder() {
@@ -345,6 +350,14 @@ public class ParquetProperties {
return statisticsEnabled;
}
+ public boolean getSizeStatisticsEnabled(ColumnDescriptor column) {
+ Boolean columnSetting = sizeStatistics.getValue(column);
+ if (columnSetting != null) {
+ return columnSetting;
+ }
+ return sizeStatisticsEnabled;
+ }
+
@Override
public String toString() {
return "Parquet page size to " + getPageSizeThreshold() + '\n'
@@ -361,7 +374,9 @@ public class ParquetProperties {
+ "Bloom filter expected number of distinct values are: " +
bloomFilterNDVs + '\n'
+ "Bloom filter false positive probabilities are: " + bloomFilterFPPs
+ '\n'
+ "Page row count limit to " + getPageRowCountLimit() + '\n'
- + "Writing page checksums is: " + (getPageWriteChecksumEnabled() ?
"on" : "off");
+ + "Writing page checksums is: " + (getPageWriteChecksumEnabled() ?
"on" : "off") + '\n'
+ + "Statistics enabled: " + statisticsEnabled + '\n'
+ + "Size statistics enabled: " + sizeStatisticsEnabled;
}
public static class Builder {
@@ -378,6 +393,7 @@ public class ParquetProperties {
private int columnIndexTruncateLength =
DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
private int statisticsTruncateLength = DEFAULT_STATISTICS_TRUNCATE_LENGTH;
private boolean statisticsEnabled = DEFAULT_STATISTICS_ENABLED;
+ private boolean sizeStatisticsEnabled = DEFAULT_SIZE_STATISTICS_ENABLED;
private final ColumnProperty.Builder<Long> bloomFilterNDVs;
private final ColumnProperty.Builder<Double> bloomFilterFPPs;
private int maxBloomFilterBytes = DEFAULT_MAX_BLOOM_FILTER_BYTES;
@@ -389,6 +405,7 @@ public class ParquetProperties {
private final ColumnProperty.Builder<ByteStreamSplitMode>
byteStreamSplitEnabled;
private Map<String, String> extraMetaData = new HashMap<>();
private final ColumnProperty.Builder<Boolean> statistics;
+ private final ColumnProperty.Builder<Boolean> sizeStatistics;
private Builder() {
enableDict =
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_IS_DICTIONARY_ENABLED);
@@ -405,6 +422,7 @@ public class ParquetProperties {
numBloomFilterCandidates =
ColumnProperty.<Integer>builder().withDefaultValue(DEFAULT_BLOOM_FILTER_CANDIDATES_NUMBER);
statistics =
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_STATISTICS_ENABLED);
+ sizeStatistics =
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_SIZE_STATISTICS_ENABLED);
}
private Builder(ParquetProperties toCopy) {
@@ -428,6 +446,7 @@ public class ParquetProperties {
this.byteStreamSplitEnabled =
ColumnProperty.builder(toCopy.byteStreamSplitEnabled);
this.extraMetaData = toCopy.extraMetaData;
this.statistics = ColumnProperty.builder(toCopy.statistics);
+ this.sizeStatistics = ColumnProperty.builder(toCopy.sizeStatistics);
}
/**
@@ -693,6 +712,30 @@ public class ParquetProperties {
return this;
}
+ /**
+ * Sets whether size statistics are enabled globally. When disabled, size
statistics will not be collected
+ * for any column unless explicitly enabled for specific columns.
+ *
+ * @param enabled whether to collect size statistics globally
+ * @return this builder for method chaining
+ */
+ public Builder withSizeStatisticsEnabled(boolean enabled) {
+ this.sizeStatistics.withDefaultValue(enabled);
+ return this;
+ }
+
+ /**
+ * Sets the size statistics enabled/disabled for the specified column. All
column size statistics are enabled by default.
+ *
+ * @param columnPath the path of the column (dot-string)
+ * @param enabled whether to collect size statistics for the column
+ * @return this builder for method chaining
+ */
+ public Builder withSizeStatisticsEnabled(String columnPath, boolean
enabled) {
+ this.sizeStatistics.withValue(columnPath, enabled);
+ return this;
+ }
+
public ParquetProperties build() {
ParquetProperties properties = new ParquetProperties(this);
// we pass a constructed but uninitialized factory to ParquetProperties
above as currently
diff --git
a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
index 164287df2..95f735f75 100644
---
a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
+++
b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
@@ -37,6 +37,7 @@ class ColumnValueCollector {
private final ColumnDescriptor path;
private final boolean statisticsEnabled;
+ private final boolean sizeStatisticsEnabled;
private BloomFilterWriter bloomFilterWriter;
private BloomFilter bloomFilter;
private Statistics<?> statistics;
@@ -45,6 +46,7 @@ class ColumnValueCollector {
ColumnValueCollector(ColumnDescriptor path, BloomFilterWriter
bloomFilterWriter, ParquetProperties props) {
this.path = path;
this.statisticsEnabled = props.getStatisticsEnabled(path);
+ this.sizeStatisticsEnabled = props.getSizeStatisticsEnabled(path);
resetPageStatistics();
initBloomFilter(bloomFilterWriter, props);
}
@@ -53,8 +55,11 @@ class ColumnValueCollector {
this.statistics = statisticsEnabled
? Statistics.createStats(path.getPrimitiveType())
: Statistics.noopStats(path.getPrimitiveType());
- this.sizeStatisticsBuilder = SizeStatistics.newBuilder(
- path.getPrimitiveType(), path.getMaxRepetitionLevel(),
path.getMaxDefinitionLevel());
+ this.sizeStatisticsBuilder = sizeStatisticsEnabled
+ ? SizeStatistics.newBuilder(
+ path.getPrimitiveType(), path.getMaxRepetitionLevel(),
path.getMaxDefinitionLevel())
+ : SizeStatistics.noopBuilder(
+ path.getPrimitiveType(), path.getMaxRepetitionLevel(),
path.getMaxDefinitionLevel());
}
void writeNull(int repetitionLevel, int definitionLevel) {
diff --git
a/parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java
index 4241f23c3..97a49be65 100644
---
a/parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java
+++
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java
@@ -52,7 +52,7 @@ public class SizeStatistics {
* Builder to create a SizeStatistics.
*/
public static class Builder {
- private final PrimitiveType type;
+ protected final PrimitiveType type;
private long unencodedByteArrayDataBytes;
private final long[] repetitionLevelHistogram;
private final long[] definitionLevelHistogram;
@@ -257,4 +257,38 @@ public class SizeStatistics {
public boolean isValid() {
return valid;
}
+
+ /**
+ * Creates a no-op size statistics builder that collects no data.
+ * Used when size statistics collection is disabled.
+ */
+ private static class NoopBuilder extends Builder {
+ private NoopBuilder(PrimitiveType type, int maxRepetitionLevel, int
maxDefinitionLevel) {
+ super(type, maxRepetitionLevel, maxDefinitionLevel);
+ }
+
+ @Override
+ public void add(int repetitionLevel, int definitionLevel) {
+ // Do nothing
+ }
+
+ @Override
+ public void add(int repetitionLevel, int definitionLevel, Binary value) {
+ // Do nothing
+ }
+
+ @Override
+ public SizeStatistics build() {
+ SizeStatistics stats = new SizeStatistics(type, 0L,
Collections.emptyList(), Collections.emptyList());
+ stats.valid = false; // Mark as invalid since this is a noop builder
+ return stats;
+ }
+ }
+
+ /**
+ * Creates a builder that doesn't collect any statistics.
+ */
+ public static Builder noopBuilder(PrimitiveType type, int
maxRepetitionLevel, int maxDefinitionLevel) {
+ return new NoopBuilder(type, maxRepetitionLevel, maxDefinitionLevel);
+ }
}
diff --git a/parquet-hadoop/README.md b/parquet-hadoop/README.md
index 5817069f4..fc12a57f2 100644
--- a/parquet-hadoop/README.md
+++ b/parquet-hadoop/README.md
@@ -525,3 +525,20 @@ conf.set("parquet.column.statistics.enabled", true);
conf.set("parquet.column.statistics.enabled#column.path", false);
// The final configuration will be: Enable statistics for all columns except
'column.path'
```
+
+---
+
+**Property:** `parquet.size.statistics.enabled`
+**Description:** Whether to enable size statistics collection.
+If `true`, size statistics will be collected for all columns unless explicitly
disabled for specific columns.
+If `false`, size statistics will be disabled for all columns regardless of
column-specific settings.
+It is possible to enable or disable size statistics for specific columns by
appending `#` followed by the column path.
+**Default value:** `true`
+**Example:**
+```java
+// Enable size statistics for all columns
+conf.set("parquet.size.statistics.enabled", true);
+// Disable size statistics for 'column.path'
+conf.set("parquet.size.statistics.enabled#column.path", false);
+// The final configuration will be: Enable size statistics for all columns
except 'column.path'
+```
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
index c13781a68..f00fde9aa 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
@@ -157,6 +157,7 @@ public class ParquetOutputFormat<T> extends
FileOutputFormat<Void, T> {
public static final String PAGE_ROW_COUNT_LIMIT =
"parquet.page.row.count.limit";
public static final String PAGE_WRITE_CHECKSUM_ENABLED =
"parquet.page.write-checksum.enabled";
public static final String STATISTICS_ENABLED =
"parquet.column.statistics.enabled";
+ public static final String SIZE_STATISTICS_ENABLED =
"parquet.size.statistics.enabled";
public static JobSummaryLevel getJobSummaryLevel(Configuration conf) {
String level = conf.get(JOB_SUMMARY_LEVEL);
@@ -409,6 +410,22 @@ public class ParquetOutputFormat<T> extends
FileOutputFormat<Void, T> {
return conf.getBoolean(STATISTICS_ENABLED,
ParquetProperties.DEFAULT_STATISTICS_ENABLED);
}
+ public static void setSizeStatisticsEnabled(Configuration conf, boolean
enabled) {
+ conf.setBoolean(SIZE_STATISTICS_ENABLED, enabled);
+ }
+
+ public static void setSizeStatisticsEnabled(Configuration conf, String path,
boolean enabled) {
+ conf.setBoolean(SIZE_STATISTICS_ENABLED + "#" + path, enabled);
+ }
+
+ public static boolean getSizeStatisticsEnabled(Configuration conf) {
+ return conf.getBoolean(SIZE_STATISTICS_ENABLED,
ParquetProperties.DEFAULT_SIZE_STATISTICS_ENABLED);
+ }
+
+ public static boolean getSizeStatisticsEnabled(Configuration conf, String
path) {
+ return conf.getBoolean(SIZE_STATISTICS_ENABLED + "#" + path,
getSizeStatisticsEnabled(conf));
+ }
+
private WriteSupport<T> writeSupport;
private ParquetOutputCommitter committer;
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index f3c50c16c..d4c4bc104 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -918,6 +918,30 @@ public class ParquetWriter<T> implements Closeable {
return self();
}
+ /**
+ * Sets the size statistics enabled/disabled for the specified column. All
column size statistics are enabled by default.
+ *
+ * @param columnPath the path of the column (dot-string)
+ * @param enabled whether to collect size statistics for the column
+ * @return this builder for method chaining
+ */
+ public SELF withSizeStatisticsEnabled(String columnPath, boolean enabled) {
+ encodingPropsBuilder.withSizeStatisticsEnabled(columnPath, enabled);
+ return self();
+ }
+
+ /**
+ * Sets whether size statistics are enabled globally. When disabled, size
statistics will not be collected
+ * for any column unless explicitly enabled for specific columns.
+ *
+ * @param enabled whether to collect size statistics globally
+ * @return this builder for method chaining
+ */
+ public SELF withSizeStatisticsEnabled(boolean enabled) {
+ encodingPropsBuilder.withSizeStatisticsEnabled(enabled);
+ return self();
+ }
+
/**
* Build a {@link ParquetWriter} with the accumulated configuration.
*
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
index 55d132c19..64001bcaf 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
@@ -32,8 +32,11 @@ import static
org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESS
import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType;
import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.ImmutableMap;
@@ -524,4 +527,69 @@ public class TestParquetWriter {
assertEquals(expectedNumberOfBlocks, footer.getBlocks().size());
}
}
+
+ @Test
+ public void testSizeStatisticsControl() throws Exception {
+ MessageType schema = Types.buildMessage()
+ .required(BINARY)
+ .named("string_field")
+ .required(BOOLEAN)
+ .named("boolean_field")
+ .required(INT32)
+ .named("int32_field")
+ .named("test_schema");
+
+ SimpleGroupFactory factory = new SimpleGroupFactory(schema);
+
+ // Create test data
+ Group group = factory.newGroup()
+ .append("string_field", "test")
+ .append("boolean_field", true)
+ .append("int32_field", 42);
+
+ // Test global disable
+ File file = temp.newFile();
+ temp.delete();
+ Path path = new Path(file.getAbsolutePath());
+ try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(path)
+ .withType(schema)
+ .withSizeStatisticsEnabled(false)
+ .build()) {
+ writer.write(group);
+ }
+
+ try (ParquetFileReader reader =
ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration()))) {
+ // Verify size statistics are disabled globally
+ for (BlockMetaData block : reader.getFooter().getBlocks()) {
+ for (ColumnChunkMetaData column : block.getColumns()) {
+ assertNull(column.getSizeStatistics());
+ }
+ }
+ }
+
+ // Test column-specific control
+ file = temp.newFile();
+ temp.delete();
+ path = new Path(file.getAbsolutePath());
+ try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(path)
+ .withType(schema)
+ .withSizeStatisticsEnabled(true) // enable globally
+ .withSizeStatisticsEnabled("boolean_field", false) // disable for
specific column
+ .build()) {
+ writer.write(group);
+ }
+
+ try (ParquetFileReader reader =
ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration()))) {
+ // Verify size statistics are enabled for all columns except
boolean_field
+ for (BlockMetaData block : reader.getFooter().getBlocks()) {
+ for (ColumnChunkMetaData column : block.getColumns()) {
+ if (column.getPath().toDotString().equals("boolean_field")) {
+ assertNull(column.getSizeStatistics());
+ } else {
+ assertTrue(column.getSizeStatistics().isValid());
+ }
+ }
+ }
+ }
+ }
}