Skip to content

Commit

Permalink
fix: Added support to read parquet files with empty row groups (#6183)
Browse files Browse the repository at this point in the history
  • Loading branch information
malhotrashivam authored Oct 9, 2024
1 parent cd237b1 commit 6612f68
Show file tree
Hide file tree
Showing 7 changed files with 60 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,10 @@ private RowSet computeIndex() {

for (int rgi = 0; rgi < rowGroups.length; ++rgi) {
final long subRegionSize = rowGroups[rgi].getNum_rows();
if (subRegionSize == 0) {
// Skip empty row groups
continue;
}
final long subRegionFirstKey = (long) rgi << regionParameters.regionMaskNumBits;
final long subRegionLastKey = subRegionFirstKey + subRegionSize - 1;
sequentialBuilder.appendRange(subRegionFirstKey, subRegionLastKey);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1735,6 +1735,47 @@ public void testAllNonPartitioningColumnTypes() {
}
}

@Test
public void testReadingParquetDataWithEmptyRowGroups() {
{
// Single parquet file with empty row group
final String path =
TestParquetTools.class.getResource("/ReferenceParquetWithEmptyRowGroup1.parquet").getFile();
final Table fromDisk =
readTable(path, EMPTY.withLayout(ParquetInstructions.ParquetFileLayout.SINGLE_FILE)).select();
assertEquals(0, fromDisk.size());
assertTrue(fromDisk.getRowSet().isEmpty());
}

{
// Single parquet file with three row groups, first and third row group are non-empty, and second row group
// is empty. To generate this file, the following branch was used:
// https://github.com/malhotrashivam/deephaven-core/tree/sm-ref-branch
final String path =
TestParquetTools.class.getResource("/ReferenceParquetWithEmptyRowGroup2.parquet").getFile();
final Table fromDisk =
readTable(path, EMPTY.withLayout(ParquetInstructions.ParquetFileLayout.SINGLE_FILE)).select();
assertEquals(20, fromDisk.size());
final Table table = TableTools.emptyTable(10).update("integers = (int)(ii%3)");
final Table expected = merge(table, table);
assertTableEquals(expected, fromDisk);
}

{
// Parquet dataset with three files, first and third file have three row groups, two non-empty followed by
// an empty row group, and second file has just one empty row group.
final String dirPath = TestParquetTools.class.getResource("/datasetWithEmptyRowgroups").getFile();
assertFalse(readTable(dirPath + "/file1.parquet").isEmpty());
assertTrue(readTable(dirPath + "/file2.parquet").isEmpty());
assertFalse(readTable(dirPath + "/file3.parquet").isEmpty());

final Table table = readTable(dirPath).select();
assertEquals(2138182, table.size());
assertEquals(4, table.numColumns());
assertEquals(1068950, table.selectDistinct("price").size());
}
}

@Test
public void decimalLogicalTypeTest() {
final Table expected = TableTools.emptyTable(100_000).update(
Expand Down
Git LFS file not shown
Git LFS file not shown
Git LFS file not shown
Git LFS file not shown
Git LFS file not shown

0 comments on commit 6612f68

Please sign in to comment.