[
https://issues.apache.org/jira/browse/PARQUET-1381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17744597#comment-17744597
]
ASF GitHub Bot commented on PARQUET-1381:
-----------------------------------------
MaheshGPai commented on code in PR #1121:
URL: https://github.com/apache/parquet-mr/pull/1121#discussion_r1267997802
##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -751,6 +767,27 @@ public GroupConverter asGroupConverter() {
}
}
+ private void mergeRowGroups() throws IOException {
+ if (null == reader) {
+ return;
+ }
+
+ boolean v2EncodingHint = meta.getBlocks().stream()
+ .flatMap(b -> b.getColumns().stream())
+ .anyMatch(chunk -> {
+ EncodingStats stats = chunk.getEncodingStats();
+ return stats != null && stats.usesV2Pages();
+ });
+
+ List<ParquetFileReader> readers = new ArrayList<>();
+ do {
+ readers.add(reader);
+ initNextReader();
+ }
+ while(reader != null);
+ new RowGroupMerger(schema, newCodecName, v2EncodingHint).merge(readers,
maxRowGroupSize, writer);
Review Comment:
Yes. Underneath, it uses the same instance of ParquetFileWriter which
handles these operations.
> Add merge blocks command to parquet-tools
> -----------------------------------------
>
> Key: PARQUET-1381
> URL: https://issues.apache.org/jira/browse/PARQUET-1381
> Project: Parquet
> Issue Type: New Feature
> Components: parquet-mr
> Affects Versions: 1.10.0
> Reporter: Ekaterina Galieva
> Assignee: Ekaterina Galieva
> Priority: Major
> Labels: pull-request-available
>
> Current implementation of merge command in parquet-tools doesn't merge row
> groups, just places one after the other. Add API and command option to be
> able to merge small blocks into larger ones up to specified size limit.
> h6. Implementation details:
> Blocks are not reordered not to break possible initial predicate pushdown
> optimizations.
> Blocks are not divided to fit upper bound perfectly.
> This is an intentional performance optimization.
> This gives an opportunity to form new blocks by coping full content of
> smaller blocks by column, not by row.
> h6. Examples:
> # Input files with blocks sizes:
> {code:java}
> [128 | 35], [128 | 40], [120]{code}
> Expected output file blocks sizes:
> {{merge }}
> {code:java}
> [128 | 35 | 128 | 40 | 120]
> {code}
> {{merge -b}}
> {code:java}
> [128 | 35 | 128 | 40 | 120]
> {code}
> {{merge -b -l 256 }}
> {code:java}
> [163 | 168 | 120]
> {code}
> # Input files with blocks sizes:
> {code:java}
> [128 | 35], [40], [120], [6] {code}
> Expected output file blocks sizes:
> {{merge}}
> {code:java}
> [128 | 35 | 40 | 120 | 6]
> {code}
> {{merge -b}}
> {code:java}
> [128 | 75 | 126]
> {code}
> {{merge -b -l 256}}
> {code:java}
> [203 | 126]{code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)