|
21 | 21 | import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; |
22 | 22 | import static org.hamcrest.MatcherAssert.assertThat; |
23 | 23 | import static org.junit.Assert.assertEquals; |
| 24 | +import static org.junit.Assert.assertFalse; |
24 | 25 | import static org.junit.Assert.assertThrows; |
25 | 26 | import static org.junit.Assert.assertTrue; |
26 | 27 | import static org.mockito.Mockito.mock; |
27 | 28 | import static org.mockito.Mockito.when; |
28 | 29 |
|
29 | 30 | import java.io.ByteArrayOutputStream; |
| 31 | +import java.io.File; |
30 | 32 | import java.io.IOException; |
31 | 33 | import java.io.Serializable; |
32 | 34 | import java.io.UnsupportedEncodingException; |
|
57 | 59 | import org.apache.hadoop.conf.Configuration; |
58 | 60 | import org.apache.parquet.filter2.predicate.FilterApi; |
59 | 61 | import org.apache.parquet.filter2.predicate.FilterPredicate; |
| 62 | +import org.apache.parquet.hadoop.ParquetFileReader; |
60 | 63 | import org.apache.parquet.hadoop.ParquetInputFormat; |
61 | 64 | import org.apache.parquet.hadoop.metadata.BlockMetaData; |
| 65 | +import org.apache.parquet.hadoop.metadata.ParquetMetadata; |
| 66 | +import org.apache.parquet.hadoop.util.HadoopInputFile; |
62 | 67 | import org.apache.parquet.io.api.Binary; |
63 | 68 | import org.junit.Rule; |
64 | 69 | import org.junit.Test; |
@@ -518,6 +523,93 @@ public void testWriteAndReadFilesAsJsonForUnknownSchemaWithConfiguration() { |
518 | 523 | readPipeline.run().waitUntilFinish(); |
519 | 524 | } |
520 | 525 |
|
| 526 | + @Test |
| 527 | + public void testWriteWithDefaultWriterProperties() throws Exception { |
| 528 | + List<GenericRecord> records = generateGenericRecords(1000); |
| 529 | + |
| 530 | + mainPipeline |
| 531 | + .apply(Create.of(records).withCoder(AvroCoder.of(SCHEMA))) |
| 532 | + .apply( |
| 533 | + FileIO.<GenericRecord>write() |
| 534 | + .via(ParquetIO.sink(SCHEMA)) |
| 535 | + .to(temporaryFolder.getRoot().getAbsolutePath())); |
| 536 | + mainPipeline.run().waitUntilFinish(); |
| 537 | + |
| 538 | + File[] outputFiles = temporaryFolder.getRoot().listFiles((dir, name) -> !name.startsWith(".")); |
| 539 | + assertTrue("Expected at least one output file", outputFiles != null && outputFiles.length > 0); |
| 540 | + |
| 541 | + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(outputFiles[0].toURI()); |
| 542 | + try (ParquetFileReader reader = |
| 543 | + ParquetFileReader.open(HadoopInputFile.fromPath(hadoopPath, new Configuration()))) { |
| 544 | + ParquetMetadata footer = reader.getFooter(); |
| 545 | + |
| 546 | + // Verify bloom filters are absent by default. |
| 547 | + boolean hasBloomFilter = |
| 548 | + footer.getBlocks().stream() |
| 549 | + .flatMap(block -> block.getColumns().stream()) |
| 550 | + .anyMatch(col -> col.getBloomFilterOffset() >= 0); |
| 551 | + assertFalse("Expected no bloom filters by default", hasBloomFilter); |
| 552 | + |
| 553 | + // Verify dictionary encoding is enabled by default. |
| 554 | + boolean hasDictionary = |
| 555 | + footer.getBlocks().stream() |
| 556 | + .flatMap(block -> block.getColumns().stream()) |
| 557 | + .anyMatch(col -> col.getDictionaryPageOffset() > 0); |
| 558 | + assertTrue("Expected dictionary pages to be present by default", hasDictionary); |
| 559 | + } |
| 560 | + } |
| 561 | + |
| 562 | + @Test |
| 563 | + public void testWriteWithWriterProperties() throws Exception { |
| 564 | + List<GenericRecord> records = generateGenericRecords(1000); |
| 565 | + |
| 566 | + mainPipeline |
| 567 | + .apply(Create.of(records).withCoder(AvroCoder.of(SCHEMA))) |
| 568 | + .apply( |
| 569 | + FileIO.<GenericRecord>write() |
| 570 | + .via( |
| 571 | + ParquetIO.sink(SCHEMA) |
| 572 | + .withPageSize(256 * 1024) |
| 573 | + .withDictionaryEncoding(false) |
| 574 | + .withBloomFilterEnabled(true) |
| 575 | + .withMinRowCountForPageSizeCheck(5)) |
| 576 | + .to(temporaryFolder.getRoot().getAbsolutePath())); |
| 577 | + mainPipeline.run().waitUntilFinish(); |
| 578 | + |
| 579 | + // Read back the file metadata and verify the settings took effect. |
| 580 | + File[] outputFiles = temporaryFolder.getRoot().listFiles((dir, name) -> !name.startsWith(".")); |
| 581 | + assertTrue("Expected at least one output file", outputFiles != null && outputFiles.length > 0); |
| 582 | + |
| 583 | + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(outputFiles[0].toURI()); |
| 584 | + try (ParquetFileReader reader = |
| 585 | + ParquetFileReader.open(HadoopInputFile.fromPath(hadoopPath, new Configuration()))) { |
| 586 | + ParquetMetadata footer = reader.getFooter(); |
| 587 | + |
| 588 | + // Verify bloom filters were written: at least one column should have a bloom filter. |
| 589 | + boolean hasBloomFilter = |
| 590 | + footer.getBlocks().stream() |
| 591 | + .flatMap(block -> block.getColumns().stream()) |
| 592 | + .anyMatch(col -> col.getBloomFilterOffset() >= 0); |
| 593 | + assertTrue("Expected bloom filters to be present", hasBloomFilter); |
| 594 | + |
| 595 | + // Verify dictionary encoding was disabled: no columns should have dictionary pages. |
| 596 | + // getDictionaryPageOffset() returns 0 when no dictionary page is present. |
| 597 | + boolean hasDictionary = |
| 598 | + footer.getBlocks().stream() |
| 599 | + .flatMap(block -> block.getColumns().stream()) |
| 600 | + .anyMatch(col -> col.getDictionaryPageOffset() > 0); |
| 601 | + assertFalse( |
| 602 | + "Expected no dictionary pages when dictionary encoding is disabled", hasDictionary); |
| 603 | + } |
| 604 | + |
| 605 | + // Verify the data still round-trips correctly. |
| 606 | + PCollection<GenericRecord> readBack = |
| 607 | + readPipeline.apply( |
| 608 | + ParquetIO.read(SCHEMA).from(temporaryFolder.getRoot().getAbsolutePath() + "/*")); |
| 609 | + PAssert.that(readBack).containsInAnyOrder(records); |
| 610 | + readPipeline.run().waitUntilFinish(); |
| 611 | + } |
| 612 | + |
521 | 613 | /** Returns list of JSON representation of GenericRecords. */ |
522 | 614 | private static List<String> convertRecordsToJson(List<GenericRecord> records) { |
523 | 615 | return records.stream().map(ParseGenericRecordAsJsonFn.create()::apply).collect(toList()); |
|
0 commit comments