Skip to content

Commit d5f86d7

Browse files
authored
GH-3133: Fix SizeStatistics to handle omitted histogram (#3134)
1 parent ec537c7 commit d5f86d7

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
@@ -148,8 +148,10 @@ public SizeStatistics(
148148
List<Long> definitionLevelHistogram) {
149149
this.type = type;
150150
this.unencodedByteArrayDataBytes = unencodedByteArrayDataBytes;
151-
this.repetitionLevelHistogram = repetitionLevelHistogram;
152-
this.definitionLevelHistogram = definitionLevelHistogram;
151+
this.repetitionLevelHistogram =
152+
repetitionLevelHistogram == null ? Collections.emptyList() : repetitionLevelHistogram;
153+
this.definitionLevelHistogram =
154+
definitionLevelHistogram == null ? Collections.emptyList() : definitionLevelHistogram;
153155
}
154156

155157
/**

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
@@ -125,4 +125,20 @@ public void testCopyStatistics() {
125125
Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getRepetitionLevelHistogram());
126126
Assert.assertEquals(Arrays.asList(1L, 1L, 1L), copy.getDefinitionLevelHistogram());
127127
}
128+
129+
@Test
130+
public void testOmittedHistogram() {
131+
PrimitiveType type = Types.optional(PrimitiveType.PrimitiveTypeName.BINARY)
132+
.as(LogicalTypeAnnotation.stringType())
133+
.named("a");
134+
SizeStatistics statistics = new SizeStatistics(type, 1024L, null, null);
135+
Assert.assertEquals(Optional.of(1024L), statistics.getUnencodedByteArrayDataBytes());
136+
Assert.assertEquals(Collections.emptyList(), statistics.getRepetitionLevelHistogram());
137+
Assert.assertEquals(Collections.emptyList(), statistics.getDefinitionLevelHistogram());
138+
139+
SizeStatistics copy = statistics.copy();
140+
Assert.assertEquals(Optional.of(1024L), copy.getUnencodedByteArrayDataBytes());
141+
Assert.assertEquals(Collections.emptyList(), copy.getRepetitionLevelHistogram());
142+
Assert.assertEquals(Collections.emptyList(), copy.getDefinitionLevelHistogram());
143+
}
128144
}

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)