|
25 | 25 | import java.io.IOException; |
26 | 26 | import java.nio.charset.Charset; |
27 | 27 | import java.util.ArrayList; |
| 28 | +import java.util.Arrays; |
28 | 29 | import java.util.HashMap; |
29 | 30 | import java.util.HashSet; |
30 | 31 | import java.util.LinkedHashSet; |
31 | 32 | import java.util.List; |
32 | 33 | import java.util.Map; |
33 | 34 | import java.util.Map.Entry; |
| 35 | +import java.util.Optional; |
34 | 36 | import java.util.Set; |
35 | 37 |
|
36 | 38 | import org.apache.hadoop.conf.Configuration; |
|
41 | 43 | import org.apache.parquet.Preconditions; |
42 | 44 | import org.apache.parquet.Strings; |
43 | 45 | import org.apache.parquet.Version; |
| 46 | +import org.apache.parquet.bytes.ByteBufferAllocator; |
44 | 47 | import org.apache.parquet.bytes.BytesInput; |
45 | 48 | import org.apache.parquet.bytes.BytesUtils; |
| 49 | +import org.apache.parquet.bytes.HeapByteBufferAllocator; |
46 | 50 | import org.apache.parquet.column.ColumnDescriptor; |
| 51 | +import org.apache.parquet.column.ColumnReader; |
| 52 | +import org.apache.parquet.column.ColumnWriter; |
47 | 53 | import org.apache.parquet.column.Encoding; |
48 | 54 | import org.apache.parquet.column.EncodingStats; |
| 55 | +import org.apache.parquet.column.ParquetProperties; |
| 56 | +import org.apache.parquet.column.impl.ColumnReadStoreImpl; |
| 57 | +import org.apache.parquet.column.impl.ColumnWriteStoreV1; |
49 | 58 | import org.apache.parquet.column.page.DictionaryPage; |
| 59 | +import org.apache.parquet.column.page.PageReader; |
50 | 60 | import org.apache.parquet.column.statistics.Statistics; |
| 61 | +import org.apache.parquet.example.DummyRecordConverter; |
51 | 62 | import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; |
52 | 63 | import org.apache.parquet.hadoop.metadata.ColumnPath; |
53 | 64 | import org.apache.parquet.format.converter.ParquetMetadataConverter; |
|
57 | 68 | import org.apache.parquet.hadoop.metadata.FileMetaData; |
58 | 69 | import org.apache.parquet.hadoop.metadata.GlobalMetaData; |
59 | 70 | import org.apache.parquet.hadoop.metadata.ParquetMetadata; |
| 71 | +import org.apache.parquet.hadoop.util.BlocksCombiner; |
60 | 72 | import org.apache.parquet.hadoop.util.HadoopOutputFile; |
61 | 73 | import org.apache.parquet.hadoop.util.HadoopStreams; |
62 | 74 | import org.apache.parquet.io.InputFile; |
@@ -519,6 +531,106 @@ public void appendFile(InputFile file) throws IOException { |
519 | 531 | ParquetFileReader.open(file).appendTo(this); |
520 | 532 | } |
521 | 533 |
|
| 534 | + public int merge(List<InputFile> inputFiles, CodecFactory.BytesCompressor compressor, String createdBy, long maxBlockSize) throws IOException { |
| 535 | + List<ParquetFileReader> readers = getReaders(inputFiles); |
| 536 | + ByteBufferAllocator allocator = new HeapByteBufferAllocator(); |
| 537 | + ColumnReadStoreImpl columnReadStore = new ColumnReadStoreImpl(null, new DummyRecordConverter(schema).getRootConverter(), schema, createdBy); |
| 538 | + this.start(); |
| 539 | + List<BlocksCombiner.SmallBlocksUnion> largeBlocks = BlocksCombiner.combineLargeBlocks(readers, maxBlockSize); |
| 540 | + for (BlocksCombiner.SmallBlocksUnion smallBlocks : largeBlocks) { |
| 541 | + for (int columnIndex = 0; columnIndex < schema.getColumns().size(); columnIndex++) { |
| 542 | + ColumnDescriptor path = schema.getColumns().get(columnIndex); |
| 543 | + ColumnChunkPageWriteStore store = new ColumnChunkPageWriteStore(compressor, schema, allocator); |
| 544 | + ColumnWriteStoreV1 columnWriteStoreV1 = new ColumnWriteStoreV1(store, ParquetProperties.builder().build()); |
| 545 | + for (BlocksCombiner.SmallBlock smallBlock : smallBlocks.getBlocks()) { |
| 546 | + ParquetFileReader parquetFileReader = smallBlock.getReader(); |
| 547 | + try { |
| 548 | + Optional<PageReader> columnChunkPageReader = parquetFileReader.readColumnInBlock(smallBlock.getBlockIndex(), path); |
| 549 | + ColumnWriter columnWriter = columnWriteStoreV1.getColumnWriter(path); |
| 550 | + if (columnChunkPageReader.isPresent()) { |
| 551 | + ColumnReader columnReader = columnReadStore.newMemColumnReader(path, columnChunkPageReader.get()); |
| 552 | + for (int i = 0; i < columnReader.getTotalValueCount(); i++) { |
| 553 | + consumeTriplet(columnWriter, columnReader); |
| 554 | + } |
| 555 | + } else { |
| 556 | + MessageType inputFileSchema = parquetFileReader.getFileMetaData().getSchema(); |
| 557 | + String[] parentPath = getExisingParentPath(path, inputFileSchema); |
| 558 | + int def = parquetFileReader.getFileMetaData().getSchema().getMaxDefinitionLevel(parentPath); |
| 559 | + int rep = parquetFileReader.getFileMetaData().getSchema().getMaxRepetitionLevel(parentPath); |
| 560 | + for (int i = 0; i < parquetFileReader.getBlockMetaData(smallBlock.getBlockIndex()).getRowCount(); i++) { |
| 561 | + columnWriter.writeNull(rep, def); |
| 562 | + } |
| 563 | + } |
| 564 | + } catch (Exception e) { |
| 565 | + LOG.error("File {} is not readable", parquetFileReader.getFile(), e); |
| 566 | + } |
| 567 | + } |
| 568 | + if (columnIndex == 0) { |
| 569 | + this.startBlock(smallBlocks.getRowCount()); |
| 570 | + } |
| 571 | + columnWriteStoreV1.flush(); |
| 572 | + store.flushToFileWriter(path, this); |
| 573 | + } |
| 574 | + this.endBlock(); |
| 575 | + } |
| 576 | + this.end(new HashMap<>()); |
| 577 | + |
| 578 | + BlocksCombiner.closeReaders(readers); |
| 579 | + return 0; |
| 580 | + } |
| 581 | + |
| 582 | + private String[] getExisingParentPath(ColumnDescriptor path, MessageType inputFileSchema) { |
| 583 | + List<String> parentPath = Arrays.asList(path.getPath()); |
| 584 | + while (parentPath.size() > 0 && !inputFileSchema.containsPath(parentPath.toArray(new String[parentPath.size()]))) { |
| 585 | + parentPath = parentPath.subList(0, parentPath.size() - 1); |
| 586 | + } |
| 587 | + return parentPath.toArray(new String[parentPath.size()]); |
| 588 | + } |
| 589 | + |
| 590 | + private List<ParquetFileReader> getReaders(List<InputFile> inputFiles) throws IOException { |
| 591 | + List<ParquetFileReader> readers = new ArrayList<>(); |
| 592 | + for (InputFile inputFile : inputFiles) { |
| 593 | + readers.add(ParquetFileReader.open(inputFile)); |
| 594 | + } |
| 595 | + return readers; |
| 596 | + } |
| 597 | + |
| 598 | + private void consumeTriplet(ColumnWriter columnWriter, ColumnReader columnReader) { |
| 599 | + int definitionLevel = columnReader.getCurrentDefinitionLevel(); |
| 600 | + int repetitionLevel = columnReader.getCurrentRepetitionLevel(); |
| 601 | + ColumnDescriptor column = columnReader.getDescriptor(); |
| 602 | + PrimitiveType type = column.getPrimitiveType(); |
| 603 | + if (definitionLevel < column.getMaxDefinitionLevel()) { |
| 604 | + columnWriter.writeNull(repetitionLevel, definitionLevel); |
| 605 | + } else { |
| 606 | + switch (type.getPrimitiveTypeName()) { |
| 607 | + case INT32: |
| 608 | + columnWriter.write(columnReader.getInteger(), repetitionLevel, definitionLevel); |
| 609 | + break; |
| 610 | + case INT64: |
| 611 | + columnWriter.write(columnReader.getLong(), repetitionLevel, definitionLevel); |
| 612 | + break; |
| 613 | + case BINARY: |
| 614 | + case FIXED_LEN_BYTE_ARRAY: |
| 615 | + case INT96: |
| 616 | + columnWriter.write(columnReader.getBinary(), repetitionLevel, definitionLevel); |
| 617 | + break; |
| 618 | + case BOOLEAN: |
| 619 | + columnWriter.write(columnReader.getBoolean(), repetitionLevel, definitionLevel); |
| 620 | + break; |
| 621 | + case FLOAT: |
| 622 | + columnWriter.write(columnReader.getFloat(), repetitionLevel, definitionLevel); |
| 623 | + break; |
| 624 | + case DOUBLE: |
| 625 | + columnWriter.write(columnReader.getDouble(), repetitionLevel, definitionLevel); |
| 626 | + break; |
| 627 | + default: |
| 628 | + throw new IllegalArgumentException("Unknown primitive type " + type); |
| 629 | + } |
| 630 | + } |
| 631 | + columnReader.consume(); |
| 632 | + } |
| 633 | + |
522 | 634 | /** |
523 | 635 | * @param file a file stream to read from |
524 | 636 | * @param rowGroups row groups to copy |
|
0 commit comments