[ https://issues.apache.org/jira/browse/PARQUET-1381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17744595#comment-17744595 ]
ASF GitHub Bot commented on PARQUET-1381: ----------------------------------------- MaheshGPai commented on code in PR #1121: URL: https://github.com/apache/parquet-mr/pull/1121#discussion_r1267995725 ########## parquet-cli/src/main/java/org/apache/parquet/cli/commands/RewriteCommand.java: ########## @@ -108,6 +120,14 @@ private RewriteOptions buildOptionsOrFail() { builder.transform(codecName); } + if (mergeRowGroups) { + Preconditions.checkArgument(maxRowGroupSize > 0, + "If merge rowgroup is enabled, max rowgroups size should be specified"); + Preconditions.checkArgument(null != codec, + "If merge rowgroup is enabled, new compression codec needs to be specified"); + builder.enableRowGroupMerge(); + builder.maxRowGroupSize(maxRowGroupSize); Review Comment: I have made changes as per the comment. I'm fine either way. ########## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/RowGroupMerger.java: ########## @@ -0,0 +1,652 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.parquet.hadoop; + +import static java.lang.String.format; +import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL; +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.parquet.column.ValuesType.VALUES; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.*; Review Comment: Corrected > 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)