Skip to content

Commit 5dcab58

Browse files
committed
apacheGH-3133: Fix SizeStatistics to handle omitted histogram (apache#3134)
1 parent de3c2d0 commit 5dcab58

File tree

3 files changed

+28
-4
lines changed

3 files changed

+28
-4
lines changed

parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -136,8 +136,10 @@ public SizeStatistics(
136136
List<Long> definitionLevelHistogram) {
137137
this.type = type;
138138
this.unencodedByteArrayDataBytes = unencodedByteArrayDataBytes;
139-
this.repetitionLevelHistogram = repetitionLevelHistogram;
140-
this.definitionLevelHistogram = definitionLevelHistogram;
139+
this.repetitionLevelHistogram =
140+
repetitionLevelHistogram == null ? Collections.emptyList() : repetitionLevelHistogram;
141+
this.definitionLevelHistogram =
142+
definitionLevelHistogram == null ? Collections.emptyList() : definitionLevelHistogram;
141143
}
142144

143145
/**

parquet-column/src/test/java/org/apache/parquet/column/statistics/TestSizeStatistics.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -124,4 +124,20 @@ public void testCopyStatistics() {
124124
Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getRepetitionLevelHistogram());
125125
Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getDefinitionLevelHistogram());
126126
}
127+
128+
@Test
129+
public void testOmittedHistogram() {
130+
PrimitiveType type = Types.optional(PrimitiveType.PrimitiveTypeName.BINARY)
131+
.as(LogicalTypeAnnotation.stringType())
132+
.named("a");
133+
SizeStatistics statistics = new SizeStatistics(type, 1024L, null, null);
134+
Assert.assertEquals(Optional.of(1024L), statistics.getUnencodedByteArrayDataBytes());
135+
Assert.assertEquals(Collections.emptyList(), statistics.getRepetitionLevelHistogram());
136+
Assert.assertEquals(Collections.emptyList(), statistics.getDefinitionLevelHistogram());
137+
138+
SizeStatistics copy = statistics.copy();
139+
Assert.assertEquals(Optional.of(1024L), copy.getUnencodedByteArrayDataBytes());
140+
Assert.assertEquals(Collections.emptyList(), copy.getRepetitionLevelHistogram());
141+
Assert.assertEquals(Collections.emptyList(), copy.getDefinitionLevelHistogram());
142+
}
127143
}

parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2382,8 +2382,14 @@ public static SizeStatistics toParquetSizeStatistics(org.apache.parquet.column.s
23822382
formatStats.setUnencoded_byte_array_data_bytes(
23832383
stats.getUnencodedByteArrayDataBytes().get());
23842384
}
2385-
formatStats.setRepetition_level_histogram(stats.getRepetitionLevelHistogram());
2386-
formatStats.setDefinition_level_histogram(stats.getDefinitionLevelHistogram());
2385+
List<Long> repLevelHistogram = stats.getRepetitionLevelHistogram();
2386+
if (repLevelHistogram != null && !repLevelHistogram.isEmpty()) {
2387+
formatStats.setRepetition_level_histogram(repLevelHistogram);
2388+
}
2389+
List<Long> defLevelHistogram = stats.getDefinitionLevelHistogram();
2390+
if (defLevelHistogram != null && !defLevelHistogram.isEmpty()) {
2391+
formatStats.setDefinition_level_histogram(defLevelHistogram);
2392+
}
23872393
return formatStats;
23882394
}
23892395
}

0 commit comments

Comments
 (0)