Skip to content

Commit

Permalink
GH-3123: Omit level histogram for some max levels (#3124)
Browse files Browse the repository at this point in the history
  • Loading branch information
wgtmac authored Jan 21, 2025
1 parent 8127cc3 commit ec537c7
Show file tree
Hide file tree
Showing 3 changed files with 102 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,16 @@ public static class Builder {
private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLevel) {
this.type = type;
this.unencodedByteArrayDataBytes = 0L;
repetitionLevelHistogram = new long[maxRepetitionLevel + 1];
definitionLevelHistogram = new long[maxDefinitionLevel + 1];
if (maxRepetitionLevel > 0) {
repetitionLevelHistogram = new long[maxRepetitionLevel + 1];
} else {
repetitionLevelHistogram = new long[0]; // omitted
}
if (maxDefinitionLevel > 1) {
definitionLevelHistogram = new long[maxDefinitionLevel + 1];
} else {
definitionLevelHistogram = new long[0]; // omitted
}
}

/**
Expand All @@ -79,8 +87,12 @@ private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLev
* @param definitionLevel definition level of the value
*/
public void add(int repetitionLevel, int definitionLevel) {
repetitionLevelHistogram[repetitionLevel]++;
definitionLevelHistogram[definitionLevel]++;
if (repetitionLevelHistogram.length > 0) {
repetitionLevelHistogram[repetitionLevel]++;
}
if (definitionLevelHistogram.length > 0) {
definitionLevelHistogram[definitionLevel]++;
}
}

/**
Expand Down Expand Up @@ -160,16 +172,29 @@ public void mergeStatistics(SizeStatistics other) {

Preconditions.checkArgument(type.equals(other.type), "Cannot merge SizeStatistics of different types");
unencodedByteArrayDataBytes = Math.addExact(unencodedByteArrayDataBytes, other.unencodedByteArrayDataBytes);
for (int i = 0; i < repetitionLevelHistogram.size(); i++) {
repetitionLevelHistogram.set(
i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i)));

if (other.repetitionLevelHistogram.isEmpty()) {
repetitionLevelHistogram.clear();
} else {
Preconditions.checkArgument(
repetitionLevelHistogram.size() == other.repetitionLevelHistogram.size(),
"Cannot merge SizeStatistics with different repetition level histogram size");
for (int i = 0; i < repetitionLevelHistogram.size(); i++) {
repetitionLevelHistogram.set(
i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i)));
}
}
for (int i = 0; i < definitionLevelHistogram.size(); i++) {
definitionLevelHistogram.set(
i,
Math.addExact(
definitionLevelHistogram.get(i),
other.getDefinitionLevelHistogram().get(i)));

if (other.definitionLevelHistogram.isEmpty()) {
definitionLevelHistogram.clear();
} else {
Preconditions.checkArgument(
definitionLevelHistogram.size() == other.definitionLevelHistogram.size(),
"Cannot merge SizeStatistics with different definition level histogram size");
for (int i = 0; i < definitionLevelHistogram.size(); i++) {
definitionLevelHistogram.set(
i, Math.addExact(definitionLevelHistogram.get(i), other.definitionLevelHistogram.get(i)));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.parquet.column.statistics;

import java.util.Arrays;
import java.util.Collections;
import java.util.Optional;
import org.apache.parquet.io.api.Binary;
import org.apache.parquet.schema.LogicalTypeAnnotation;
Expand Down Expand Up @@ -67,7 +68,7 @@ public void testAddNonBinaryType() {
SizeStatistics statistics = builder.build();
Assert.assertEquals(Optional.empty(), statistics.getUnencodedByteArrayDataBytes());
Assert.assertEquals(Arrays.asList(2L, 4L), statistics.getRepetitionLevelHistogram());
Assert.assertEquals(Arrays.asList(3L, 3L), statistics.getDefinitionLevelHistogram());
Assert.assertEquals(Collections.emptyList(), statistics.getDefinitionLevelHistogram());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
Expand Down Expand Up @@ -82,12 +83,12 @@ public void testBinaryColumnSizeStatistics() throws IOException {

SizeStatistics sizeStatistics = column.getSizeStatistics();
Assert.assertEquals(Optional.of(4L), sizeStatistics.getUnencodedByteArrayDataBytes());
Assert.assertEquals(Arrays.asList(4L), sizeStatistics.getRepetitionLevelHistogram());
Assert.assertEquals(Arrays.asList(0L, 4L), sizeStatistics.getDefinitionLevelHistogram());
Assert.assertEquals(Collections.emptyList(), sizeStatistics.getRepetitionLevelHistogram());
Assert.assertEquals(Collections.emptyList(), sizeStatistics.getDefinitionLevelHistogram());

ColumnIndex columnIndex = reader.readColumnIndex(column);
Assert.assertEquals(Arrays.asList(2L, 2L), columnIndex.getRepetitionLevelHistogram());
Assert.assertEquals(Arrays.asList(0L, 2L, 0L, 2L), columnIndex.getDefinitionLevelHistogram());
Assert.assertEquals(Collections.emptyList(), columnIndex.getRepetitionLevelHistogram());
Assert.assertEquals(Collections.emptyList(), columnIndex.getDefinitionLevelHistogram());

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

@Test
public void testNestedRepeatedOptionalColumnSizeStatistics() throws IOException {
MessageType schema = Types.buildMessage()
.optionalGroup()
.repeatedGroup()
.optional(PrimitiveType.PrimitiveTypeName.BINARY)
.as(LogicalTypeAnnotation.stringType())
.named("value")
.named("list")
.named("outer")
.named("msg");

Configuration conf = new Configuration();
GroupWriteSupport.setSchema(schema, conf);

GroupFactory factory = new SimpleGroupFactory(schema);
Path path = newTempPath();
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(path)
.withPageRowCountLimit(2)
.withMinRowCountForPageSizeCheck(1)
.withConf(conf)
.build()) {
// Create groups with different nesting patterns
Group g1 = factory.newGroup();
Group outer1 = g1.addGroup("outer");
Group list1 = outer1.addGroup("list");
list1.append("value", "a");
Group list2 = outer1.addGroup("list");
list2.append("value", "b");
writer.write(g1);

Group g2 = factory.newGroup();
Group outer2 = g2.addGroup("outer");
Group list3 = outer2.addGroup("list");
list3.append("value", "c");
writer.write(g2);
}

try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration()))) {
ParquetMetadata footer = reader.getFooter();
ColumnChunkMetaData column = footer.getBlocks().get(0).getColumns().get(0);

SizeStatistics sizeStatistics = column.getSizeStatistics();
Assert.assertEquals(Optional.of(3L), sizeStatistics.getUnencodedByteArrayDataBytes());
Assert.assertEquals(Arrays.asList(2L, 1L), sizeStatistics.getRepetitionLevelHistogram());
Assert.assertEquals(Arrays.asList(0L, 0L, 0L, 3L), sizeStatistics.getDefinitionLevelHistogram());

ColumnIndex columnIndex = reader.readColumnIndex(column);
Assert.assertEquals(Arrays.asList(2L, 1L), sizeStatistics.getRepetitionLevelHistogram());
Assert.assertEquals(Arrays.asList(0L, 0L, 0L, 3L), sizeStatistics.getDefinitionLevelHistogram());

OffsetIndex offsetIndex = reader.readOffsetIndex(column);
Assert.assertEquals(1, offsetIndex.getPageCount());
Assert.assertEquals(Optional.of(3L), offsetIndex.getUnencodedByteArrayDataBytes(0));
}
}

private Path newTempPath() throws IOException {
File file = temp.newFile();
Preconditions.checkArgument(file.delete(), "Could not remove temp file");
Expand Down

0 comments on commit ec537c7

Please sign in to comment.