You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "advancedxy (via GitHub)" <gi...@apache.org> on 2023/07/20 09:11:32 UTC

[GitHub] [parquet-mr] advancedxy commented on a diff in pull request #1121: PARQUET-1381: Support merging of rowgroups during file rewrite

advancedxy commented on code in PR #1121:
URL: https://github.com/apache/parquet-mr/pull/1121#discussion_r1269155063


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -751,6 +764,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();

Review Comment:
   Looks like `v2EncodingHint` only checks the first parquet file..
   
   Should all the files to be checked?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RowGroupMerger.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.rewrite;
+
+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.List;
+import java.util.Collections;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.AbstractMap.SimpleEntry;
+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.CodecFactory;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+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.ParquetDecodingException;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+
+class RowGroupMerger {
+
+  private final MessageType schema;
+  private final CompressionCodecFactory.BytesInputCompressor compressor;
+  private final ParquetProperties parquetProperties;
+
+  public 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
+   */
+  public void merge(List<ParquetFileReader> inputFiles, final long maxRowGroupSize,
+                    ParquetFileWriter writer) throws IOException {
+
+    SizeEstimator estimator = new SizeEstimator(compressor.getCodecName() != CompressionCodecName.UNCOMPRESSED);
+    MutableMergedBlock mergedBlock = null;
+    for (ParquetFileReader reader : inputFiles) {
+      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.getCompressedSize() + 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.getCompressedSize();
+        mergedBlock.merge(blockMeta, group);
+        //update our estimator
+        long currentBlockEffect = mergedBlock.getCompressedSize() - sizeBeforeMerge;
+        estimator.update(currentBlockEffect, blockMeta);
+      }
+    }
+    if (mergedBlock != null) {
+      saveBlockTo(mergedBlock, writer);
+    }
+    mergedBlock = null;
+  }
+
+
+  private 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 {
+      //we copy as some compressors use shared memory
+      return BytesInput.copy(compressor.compress(bytes));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private 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);
+        }
+        return null;
+      }
+
+      @Override
+      public Void visit(DataPageV2 page) {
+        try {
+          writer.writeDataPageV2(page.getRowCount(), page.getNullCount(), page.getValueCount(),
+            page.getRepetitionLevels(), page.getDefinitionLevels(), page.getDataEncoding(),
+            page.getData(), page.getUncompressedSize(), page.getStatistics());
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+        return null;
+      }
+    });
+  }
+
+  private static DictionaryPage getCompressedDictionary(DictionaryPage dictionary, CompressionCodecFactory.BytesInputCompressor compressor) {
+    return new DictionaryPage(
+      compress(dictionary.getBytes(), compressor),
+      dictionary.getUncompressedSize(),
+      dictionary.getDictionarySize(),
+      dictionary.getEncoding());
+  }
+
+  private interface MergedBlock {
+    long rowCount();
+
+    List<MergedColumn> columnsInOrder();
+  }
+
+  private interface MergedColumn {
+    long getValueCount();
+
+    CompressionCodecName getCompression();
+
+    ColumnDescriptor getColumnDesc();
+
+    void writeDataPagesTo(ParquetFileWriter writer);
+
+    void writeDictionaryPageTo(ParquetFileWriter writer) throws IOException;
+  }
+
+  private class MutableMergedBlock implements MergedBlock {
+
+    private final Map<ColumnDescriptor, MutableMergedColumn> columns = new HashMap<>();
+    private final MessageType schema;
+    private long recordCount;
+    private long compressedSize;
+
+    private MutableMergedBlock(MessageType schema) {
+      this.schema = schema;
+    }
+
+    @Override
+    public long rowCount() {
+      return recordCount;
+    }
+
+    private long getCompressedSize() {
+      return compressedSize;
+    }
+
+    @Override
+    public List<MergedColumn> columnsInOrder() {
+      return schema.getColumns()
+        .stream()
+        .map(columns::get)
+        .collect(Collectors.toList());
+    }
+
+    MutableMergedColumn getOrCreateColumn(ColumnDescriptor column) {
+      return columns.computeIfAbsent(column, desc -> new MutableMergedColumn(desc, this::addCompressedBytes));
+    }
+
+    void addRowCount(long rowCount) {
+      recordCount += rowCount;
+    }
+
+    void addCompressedBytes(long size) {
+      compressedSize += size;
+    }
+
+    void merge(BlockMetaData blockMeta, PageReadStore group) throws IOException {
+      for (Entry<ColumnDescriptor, ColumnChunkMetaData> col : getColumnsInOrder(blockMeta, schema)) {
+
+        MutableMergedColumn column = getOrCreateColumn(col.getKey());
+        PageReader columnReader = group.getPageReader(col.getKey());
+
+        DictionaryPage dictPage = columnReader.readDictionaryPage();
+        Dictionary decodedDictionary = null;
+        if (dictPage != null) {
+          decodedDictionary = dictPage.getEncoding().initDictionary(column.getColumnDesc(), dictPage);
+        }

Review Comment:
   If I understand the process of page encoding correctly: parquet tries to use dictionary encoding by default, If the dictionary grows too big, whether in size or number of distinct values, the encoding will fall back to the plain encoding. The check and fallback logic happens when emit the first page. 
   
   So when we are merging multiple column chunks from different row groups, if the first column chunks is  dictionary encoded and others are not because it fallbacks to plain encoding, we should disable the dictionary encoding for that column on purpose to avoid introducing overhead.
   
   Current logic doesn't handle that, it will use dictionary encoding if the column chunk in the first row group to be merged use dictionary encoding.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RowGroupMerger.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.rewrite;
+
+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.List;
+import java.util.Collections;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.AbstractMap.SimpleEntry;
+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.CodecFactory;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+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.ParquetDecodingException;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+
+class RowGroupMerger {
+
+  private final MessageType schema;
+  private final CompressionCodecFactory.BytesInputCompressor compressor;
+  private final ParquetProperties parquetProperties;
+
+  public 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
+   */
+  public void merge(List<ParquetFileReader> inputFiles, final long maxRowGroupSize,
+                    ParquetFileWriter writer) throws IOException {
+
+    SizeEstimator estimator = new SizeEstimator(compressor.getCodecName() != CompressionCodecName.UNCOMPRESSED);
+    MutableMergedBlock mergedBlock = null;
+    for (ParquetFileReader reader : inputFiles) {
+      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.getCompressedSize() + 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);

Review Comment:
   I checked related code, seems that `startColumn` and `endColumn` doesn't maintain bloom filter....
   
   It might be hard to maintain bloom filters when merging multiple row groups, but it should be possible and easy to maintain bloom filter for only one row group. See ParquetWriter#L337 for related code.



-- 
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.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

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