Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ class ParquetWriter<T> implements FileAppender<T>, Closeable {
private ColumnWriteStore writeStore;
private long recordCount = 0;
private long nextCheckRecordCount = 10;
private long uncompressedBufferedSize = 0;
private boolean closed;
private ParquetFileWriter writer;
private int rowGroupOrdinal;
Expand Down Expand Up @@ -135,7 +136,9 @@ private void ensureWriterInitialized() {
@Override
public void add(T value) {
recordCount += 1;
long sizeBefore = writeStore.getBufferedSize();
model.write(0, value);
uncompressedBufferedSize += writeStore.getBufferedSize() - sizeBefore;
writeStore.endRecord();
checkSize();
}
Expand Down Expand Up @@ -191,7 +194,7 @@ public List<Long> splitOffsets() {

private void checkSize() {
if (recordCount >= nextCheckRecordCount) {
long bufferedSize = writeStore.getBufferedSize();
long bufferedSize = uncompressedBufferedSize;
double avgRecordSize = ((double) bufferedSize) / recordCount;

if (bufferedSize > (targetRowGroupSize - 2 * avgRecordSize)) {
Expand Down Expand Up @@ -234,6 +237,7 @@ private void startRowGroup() {
Math.max(recordCount / 2, props.getMinRowCountForPageSizeCheck()),
props.getMaxRowCountForPageSizeCheck());
this.recordCount = 0;
this.uncompressedBufferedSize = 0;

this.pageStore =
new ColumnChunkPageWriteStore(
Expand Down
45 changes: 45 additions & 0 deletions parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.iceberg.Files.localInput;
import static org.apache.iceberg.TableProperties.PARQUET_COLUMN_STATS_ENABLED_PREFIX;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
Expand Down Expand Up @@ -338,6 +339,50 @@ public void testAvroWriterRejectsVariantType() {
.hasMessage("Avro writer does not support variant types");
}

@Test
public void testRowGroupSizeEnforcedWithCompression() throws IOException {
// Regression test for #16325: with compression enabled, row group size check used compressed
// bytes, causing row groups to grow unbounded.
Schema schema = new Schema(optional(1, "stringCol", Types.StringType.get()));

File file = createTempFile(temp);

// Write data that is highly compressible but unique enough to avoid dictionary encoding
// from collapsing it. Each record has a unique prefix + repeated padding.
int rowGroupSize = 64 * 1024; // 64 KB
int pageSize = 4 * 1024; // 4 KB page size to force frequent page flushes
int numRecords = 500;

List<GenericData.Record> records = Lists.newArrayListWithCapacity(numRecords);
org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct());
for (int i = 0; i < numRecords; i++) {
GenericData.Record record = new GenericData.Record(avroSchema);
// Unique prefix ensures no dictionary, repeated suffix ensures high compression
record.put("stringCol", String.format("%010d", i) + Strings.repeat("a", 1014));
records.add(record);
}

write(
file,
schema,
ImmutableMap.<String, String>builder()
.put(PARQUET_ROW_GROUP_SIZE_BYTES, Integer.toString(rowGroupSize))
.put(PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT, "1")
.put(PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT, "100")
.put(PARQUET_COMPRESSION, "gzip")
.put("write.parquet.page-size-bytes", Integer.toString(pageSize))
.buildOrThrow(),
ParquetAvroWriter::buildWriter,
records.toArray(new GenericData.Record[] {}));

try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(file)))) {
List<BlockMetaData> rowGroups = reader.getRowGroups();
// With 500 KB of uncompressed data and a 64 KB row group target,
// we must get multiple row groups
assertThat(rowGroups).hasSizeGreaterThan(1);
}
}

private Pair<File, Long> generateFile(
Function<MessageType, ParquetValueWriter<?>> createWriterFunc,
int desiredRecordCount,
Expand Down