[ 
https://issues.apache.org/jira/browse/PARQUET-1381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17088099#comment-17088099
 ] 

ASF GitHub Bot commented on PARQUET-1381:
-----------------------------------------

brimzi commented on a change in pull request #775:
URL: https://github.com/apache/parquet-mr/pull/775#discussion_r411708296



##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/RowGroupMerger.java
##########
@@ -0,0 +1,634 @@
+/*
+ * 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 static org.apache.parquet.hadoop.ParquetFileWriter.getColumnsInOrder;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.ValuesWriter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+
+class RowGroupMerger {
+
+  private final MessageType schema;
+  private final CodecFactory.BytesInputCompressor compressor;
+  private final ParquetProperties parquetProperties;
+
+  RowGroupMerger(MessageType schema, CompressionCodecName compression, boolean 
useV2ValueWriter) {
+    this(schema, new Configuration(), compression, useV2ValueWriter);
+  }
+
+  RowGroupMerger(MessageType schema, Configuration conf, CompressionCodecName 
compression, boolean useV2ValueWriter) {
+    this(schema, conf, compression, createParquetProperties(useV2ValueWriter));
+  }
+
+  RowGroupMerger(MessageType schema, Configuration conf, CompressionCodecName 
compression, ParquetProperties parquetProperties) {
+    this.schema = schema;
+    this.parquetProperties = parquetProperties;
+    this.compressor = new CodecFactory(conf, 
this.parquetProperties.getPageSizeThreshold()).getCompressor(compression);
+  }
+
+  /**
+   * Merges the row groups making sure that new row groups do not exceed the 
supplied maxRowGroupSize
+   *
+   * @param inputFiles      input files to merge
+   * @param maxRowGroupSize the max limit for new blocks
+   * @param writer          writer to write the new blocks to
+   * @throws IOException if an IO error occurs
+   */
+  void merge(List<InputFile> inputFiles, final long maxRowGroupSize, 
ParquetFileWriter writer) throws IOException {
+
+    SizeEstimator estimator = new SizeEstimator(compressor.getCodecName() != 
CompressionCodecName.UNCOMPRESSED);
+    MutableMergedBlock mergedBlock = null;
+    for (InputFile file : inputFiles) {
+      try (ParquetFileReader reader = ParquetFileReader.open(file)) {
+
+        for (BlockMetaData blockMeta : reader.getRowGroups()) {
+          PageReadStore group = reader.readNextRowGroup();
+          Preconditions.checkState(group != null,
+            "number of groups returned by FileReader does not match metadata");
+
+          if (mergedBlock != null && mergedBlock.getCompressedBytesSize() + 
estimator.estimate(blockMeta) > maxRowGroupSize) {
+            saveBlockTo(mergedBlock, writer);
+            mergedBlock = null;
+          }
+
+          if (mergedBlock == null && estimator.estimate(blockMeta) > 
maxRowGroupSize) {
+            //save it directly without re encoding it
+            saveBlockTo(ReadOnlyMergedBlock.of(blockMeta, group, schema, 
compressor), writer);
+            continue;
+          }
+
+          if (mergedBlock == null) {
+            mergedBlock = new MutableMergedBlock(schema);
+          }
+
+          long sizeBeforeMerge = mergedBlock.getCompressedBytesSize();
+          mergedBlock.merge(blockMeta, group);
+          //update our estimator
+          long currentBlockEffect = mergedBlock.getCompressedBytesSize() - 
sizeBeforeMerge;
+          estimator.update(currentBlockEffect, blockMeta);
+        }
+      }
+    }
+    if (mergedBlock != null)
+      saveBlockTo(mergedBlock, writer);
+  }
+
+
+  private static void saveBlockTo(MergedBlock block, ParquetFileWriter writer) 
{
+    try {
+      writer.startBlock(block.rowCount());
+
+      for (MergedColumn col : block.columnsInOrder()) {
+        writer.startColumn(col.getColumnDesc(), col.getValueCount(), 
col.getCompression());
+
+        col.writeDictionaryPageTo(writer);
+        col.writeDataPagesTo(writer);
+
+        writer.endColumn();
+      }
+
+      writer.endBlock();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private static ParquetProperties createParquetProperties(boolean 
useV2Writer) {
+    ParquetProperties.Builder builder = ParquetProperties.builder();
+    if (useV2Writer) {
+      builder.withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0);
+    }
+    return builder.build();
+  }
+
+  private BytesInput compress(BytesInput bytes) {
+    return compress(bytes, compressor);
+  }
+
+  private static BytesInput compress(BytesInput bytes, 
CompressionCodecFactory.BytesInputCompressor compressor) {
+    try {
+      return BytesInput.copy(compressor.compress(bytes));//we copy as some 
compressors use shared memory
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private static BiConsumer<ValuesReader, ValuesWriter> 
createWritingBridge(PrimitiveType.PrimitiveTypeName typeName) {
+    switch (typeName) {
+      case FIXED_LEN_BYTE_ARRAY:
+      case INT96:
+      case BINARY:
+        return (src, dest) -> dest.writeBytes(src.readBytes());
+      case BOOLEAN:
+        return (src, dest) -> dest.writeBoolean(src.readBoolean());
+      case DOUBLE:
+        return (src, dest) -> dest.writeDouble(src.readDouble());
+      case FLOAT:
+        return (src, dest) -> dest.writeFloat(src.readFloat());
+      case INT32:
+        return (src, dest) -> dest.writeInteger(src.readInteger());
+      case INT64:
+        return (src, dest) -> dest.writeLong(src.readLong());
+      default:
+        throw new RuntimeException("Unsupported column primitive type: " + 
typeName.name());
+    }
+  }
+
+  private static void writePageTo(DataPage dataPage, ParquetFileWriter writer) 
{
+    dataPage.accept(new DataPage.Visitor<Void>() {
+      @Override
+      public Void visit(DataPageV1 page) {
+        try {
+          if (page.getIndexRowCount().isPresent()) {
+            writer.writeDataPage(page.getValueCount(), 
page.getUncompressedSize(),
+              page.getBytes(), page.getStatistics(), 
page.getIndexRowCount().get(), page.getRlEncoding(),
+              page.getDlEncoding(), page.getValueEncoding());
+
+          } else {
+            writer.writeDataPage(page.getValueCount(), 
page.getUncompressedSize(),
+              page.getBytes(), page.getStatistics(), page.getRlEncoding(),
+              page.getDlEncoding(), page.getValueEncoding());
+          }
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);

Review comment:
       I am forced to wrap it here because in the Visitor interface this method 
does not throw any exception, hence I am supposed to swallow all checked 
exceptions in this implementation




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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.3.4#803005)

Reply via email to