Skip to content

Commit ec537c7

Browse files
authored
GH-3123: Omit level histogram for some max levels (#3124)
1 parent 8127cc3 commit ec537c7

File tree

3 files changed

+102
-18
lines changed

3 files changed

+102
-18
lines changed

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

Lines changed: 38 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -67,8 +67,16 @@ public static class Builder {
6767
private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLevel) {
6868
this.type = type;
6969
this.unencodedByteArrayDataBytes = 0L;
70-
repetitionLevelHistogram = new long[maxRepetitionLevel + 1];
71-
definitionLevelHistogram = new long[maxDefinitionLevel + 1];
70+
if (maxRepetitionLevel > 0) {
71+
repetitionLevelHistogram = new long[maxRepetitionLevel + 1];
72+
} else {
73+
repetitionLevelHistogram = new long[0]; // omitted
74+
}
75+
if (maxDefinitionLevel > 1) {
76+
definitionLevelHistogram = new long[maxDefinitionLevel + 1];
77+
} else {
78+
definitionLevelHistogram = new long[0]; // omitted
79+
}
7280
}
7381

7482
/**
@@ -79,8 +87,12 @@ private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLev
7987
* @param definitionLevel definition level of the value
8088
*/
8189
public void add(int repetitionLevel, int definitionLevel) {
82-
repetitionLevelHistogram[repetitionLevel]++;
83-
definitionLevelHistogram[definitionLevel]++;
90+
if (repetitionLevelHistogram.length > 0) {
91+
repetitionLevelHistogram[repetitionLevel]++;
92+
}
93+
if (definitionLevelHistogram.length > 0) {
94+
definitionLevelHistogram[definitionLevel]++;
95+
}
8496
}
8597

8698
/**
@@ -160,16 +172,29 @@ public void mergeStatistics(SizeStatistics other) {
160172

161173
Preconditions.checkArgument(type.equals(other.type), "Cannot merge SizeStatistics of different types");
162174
unencodedByteArrayDataBytes = Math.addExact(unencodedByteArrayDataBytes, other.unencodedByteArrayDataBytes);
163-
for (int i = 0; i < repetitionLevelHistogram.size(); i++) {
164-
repetitionLevelHistogram.set(
165-
i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i)));
175+
176+
if (other.repetitionLevelHistogram.isEmpty()) {
177+
repetitionLevelHistogram.clear();
178+
} else {
179+
Preconditions.checkArgument(
180+
repetitionLevelHistogram.size() == other.repetitionLevelHistogram.size(),
181+
"Cannot merge SizeStatistics with different repetition level histogram size");
182+
for (int i = 0; i < repetitionLevelHistogram.size(); i++) {
183+
repetitionLevelHistogram.set(
184+
i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i)));
185+
}
166186
}
167-
for (int i = 0; i < definitionLevelHistogram.size(); i++) {
168-
definitionLevelHistogram.set(
169-
i,
170-
Math.addExact(
171-
definitionLevelHistogram.get(i),
172-
other.getDefinitionLevelHistogram().get(i)));
187+
188+
if (other.definitionLevelHistogram.isEmpty()) {
189+
definitionLevelHistogram.clear();
190+
} else {
191+
Preconditions.checkArgument(
192+
definitionLevelHistogram.size() == other.definitionLevelHistogram.size(),
193+
"Cannot merge SizeStatistics with different definition level histogram size");
194+
for (int i = 0; i < definitionLevelHistogram.size(); i++) {
195+
definitionLevelHistogram.set(
196+
i, Math.addExact(definitionLevelHistogram.get(i), other.definitionLevelHistogram.get(i)));
197+
}
173198
}
174199
}
175200

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.parquet.column.statistics;
2020

2121
import java.util.Arrays;
22+
import java.util.Collections;
2223
import java.util.Optional;
2324
import org.apache.parquet.io.api.Binary;
2425
import org.apache.parquet.schema.LogicalTypeAnnotation;
@@ -67,7 +68,7 @@ public void testAddNonBinaryType() {
6768
SizeStatistics statistics = builder.build();
6869
Assert.assertEquals(Optional.empty(), statistics.getUnencodedByteArrayDataBytes());
6970
Assert.assertEquals(Arrays.asList(2L, 4L), statistics.getRepetitionLevelHistogram());
70-
Assert.assertEquals(Arrays.asList(3L, 3L), statistics.getDefinitionLevelHistogram());
71+
Assert.assertEquals(Collections.emptyList(), statistics.getDefinitionLevelHistogram());
7172
}
7273

7374
@Test

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

Lines changed: 62 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.io.File;
2222
import java.io.IOException;
2323
import java.util.Arrays;
24+
import java.util.Collections;
2425
import java.util.Optional;
2526
import org.apache.hadoop.conf.Configuration;
2627
import org.apache.hadoop.fs.Path;
@@ -82,12 +83,12 @@ public void testBinaryColumnSizeStatistics() throws IOException {
8283

8384
SizeStatistics sizeStatistics = column.getSizeStatistics();
8485
Assert.assertEquals(Optional.of(4L), sizeStatistics.getUnencodedByteArrayDataBytes());
85-
Assert.assertEquals(Arrays.asList(4L), sizeStatistics.getRepetitionLevelHistogram());
86-
Assert.assertEquals(Arrays.asList(0L, 4L), sizeStatistics.getDefinitionLevelHistogram());
86+
Assert.assertEquals(Collections.emptyList(), sizeStatistics.getRepetitionLevelHistogram());
87+
Assert.assertEquals(Collections.emptyList(), sizeStatistics.getDefinitionLevelHistogram());
8788

8889
ColumnIndex columnIndex = reader.readColumnIndex(column);
89-
Assert.assertEquals(Arrays.asList(2L, 2L), columnIndex.getRepetitionLevelHistogram());
90-
Assert.assertEquals(Arrays.asList(0L, 2L, 0L, 2L), columnIndex.getDefinitionLevelHistogram());
90+
Assert.assertEquals(Collections.emptyList(), columnIndex.getRepetitionLevelHistogram());
91+
Assert.assertEquals(Collections.emptyList(), columnIndex.getDefinitionLevelHistogram());
9192

9293
OffsetIndex offsetIndex = reader.readOffsetIndex(column);
9394
Assert.assertEquals(2, offsetIndex.getPageCount());
@@ -96,6 +97,63 @@ public void testBinaryColumnSizeStatistics() throws IOException {
9697
}
9798
}
9899

100+
@Test
101+
public void testNestedRepeatedOptionalColumnSizeStatistics() throws IOException {
102+
MessageType schema = Types.buildMessage()
103+
.optionalGroup()
104+
.repeatedGroup()
105+
.optional(PrimitiveType.PrimitiveTypeName.BINARY)
106+
.as(LogicalTypeAnnotation.stringType())
107+
.named("value")
108+
.named("list")
109+
.named("outer")
110+
.named("msg");
111+
112+
Configuration conf = new Configuration();
113+
GroupWriteSupport.setSchema(schema, conf);
114+
115+
GroupFactory factory = new SimpleGroupFactory(schema);
116+
Path path = newTempPath();
117+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(path)
118+
.withPageRowCountLimit(2)
119+
.withMinRowCountForPageSizeCheck(1)
120+
.withConf(conf)
121+
.build()) {
122+
// Create groups with different nesting patterns
123+
Group g1 = factory.newGroup();
124+
Group outer1 = g1.addGroup("outer");
125+
Group list1 = outer1.addGroup("list");
126+
list1.append("value", "a");
127+
Group list2 = outer1.addGroup("list");
128+
list2.append("value", "b");
129+
writer.write(g1);
130+
131+
Group g2 = factory.newGroup();
132+
Group outer2 = g2.addGroup("outer");
133+
Group list3 = outer2.addGroup("list");
134+
list3.append("value", "c");
135+
writer.write(g2);
136+
}
137+
138+
try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration()))) {
139+
ParquetMetadata footer = reader.getFooter();
140+
ColumnChunkMetaData column = footer.getBlocks().get(0).getColumns().get(0);
141+
142+
SizeStatistics sizeStatistics = column.getSizeStatistics();
143+
Assert.assertEquals(Optional.of(3L), sizeStatistics.getUnencodedByteArrayDataBytes());
144+
Assert.assertEquals(Arrays.asList(2L, 1L), sizeStatistics.getRepetitionLevelHistogram());
145+
Assert.assertEquals(Arrays.asList(0L, 0L, 0L, 3L), sizeStatistics.getDefinitionLevelHistogram());
146+
147+
ColumnIndex columnIndex = reader.readColumnIndex(column);
148+
Assert.assertEquals(Arrays.asList(2L, 1L), sizeStatistics.getRepetitionLevelHistogram());
149+
Assert.assertEquals(Arrays.asList(0L, 0L, 0L, 3L), sizeStatistics.getDefinitionLevelHistogram());
150+
151+
OffsetIndex offsetIndex = reader.readOffsetIndex(column);
152+
Assert.assertEquals(1, offsetIndex.getPageCount());
153+
Assert.assertEquals(Optional.of(3L), offsetIndex.getUnencodedByteArrayDataBytes(0));
154+
}
155+
}
156+
99157
private Path newTempPath() throws IOException {
100158
File file = temp.newFile();
101159
Preconditions.checkArgument(file.delete(), "Could not remove temp file");

0 commit comments

Comments
 (0)