From c5cfa2bda33fa66a0965d28d14211f7968b3a4ba Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Tue, 21 Jan 2025 13:09:47 +0800 Subject: [PATCH] GH-3133: Fix SizeStatistics to handle omitted histogram --- .../column/statistics/SizeStatistics.java | 6 ++++-- .../column/statistics/TestSizeStatistics.java | 16 ++++++++++++++++ .../converter/ParquetMetadataConverter.java | 10 ++++++++-- 3 files changed, 28 insertions(+), 4 deletions(-) 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 d267d3f6d0..fe1826c194 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 @@ -148,8 +148,10 @@ public SizeStatistics( List definitionLevelHistogram) { this.type = type; this.unencodedByteArrayDataBytes = unencodedByteArrayDataBytes; - this.repetitionLevelHistogram = repetitionLevelHistogram; - this.definitionLevelHistogram = definitionLevelHistogram; + this.repetitionLevelHistogram = + repetitionLevelHistogram == null ? Collections.emptyList() : repetitionLevelHistogram; + this.definitionLevelHistogram = + definitionLevelHistogram == null ? Collections.emptyList() : definitionLevelHistogram; } /** diff --git a/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestSizeStatistics.java b/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestSizeStatistics.java index 32bdf54f50..813298c2b6 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestSizeStatistics.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestSizeStatistics.java @@ -125,4 +125,20 @@ public void testCopyStatistics() { Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getRepetitionLevelHistogram()); Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getDefinitionLevelHistogram()); } + + @Test + public void testOmittedHistogram() { + PrimitiveType type = Types.optional(PrimitiveType.PrimitiveTypeName.BINARY) + .as(LogicalTypeAnnotation.stringType()) + .named("a"); + SizeStatistics statistics = new SizeStatistics(type, 1024L, null, null); + Assert.assertEquals(Optional.of(1024L), statistics.getUnencodedByteArrayDataBytes()); + Assert.assertEquals(Collections.emptyList(), statistics.getRepetitionLevelHistogram()); + Assert.assertEquals(Collections.emptyList(), statistics.getDefinitionLevelHistogram()); + + SizeStatistics copy = statistics.copy(); + Assert.assertEquals(Optional.of(1024L), copy.getUnencodedByteArrayDataBytes()); + Assert.assertEquals(Collections.emptyList(), copy.getRepetitionLevelHistogram()); + Assert.assertEquals(Collections.emptyList(), copy.getDefinitionLevelHistogram()); + } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java index d1c6b01c93..e72f2c33a2 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java @@ -2382,8 +2382,14 @@ public static SizeStatistics toParquetSizeStatistics(org.apache.parquet.column.s formatStats.setUnencoded_byte_array_data_bytes( stats.getUnencodedByteArrayDataBytes().get()); } - formatStats.setRepetition_level_histogram(stats.getRepetitionLevelHistogram()); - formatStats.setDefinition_level_histogram(stats.getDefinitionLevelHistogram()); + List repLevelHistogram = stats.getRepetitionLevelHistogram(); + if (repLevelHistogram != null && !repLevelHistogram.isEmpty()) { + formatStats.setRepetition_level_histogram(repLevelHistogram); + } + List defLevelHistogram = stats.getDefinitionLevelHistogram(); + if (defLevelHistogram != null && !defLevelHistogram.isEmpty()) { + formatStats.setDefinition_level_histogram(defLevelHistogram); + } return formatStats; } }