You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/12/14 07:47:30 UTC

[GitHub] [parquet-mr] wgtmac opened a new pull request, #1014: PARQUET-2075: Implement ParquetRewriter

wgtmac opened a new pull request, #1014:
URL: https://github.com/apache/parquet-mr/pull/1014

   ### Jira
   
   - This patch aims to solve the first step of [PARQUET-2075](https://issues.apache.org/jira/browse/PARQUET-2075).
   
   ### Tests
   
   - Make sure all tasks pass, especially CompressionConverterTest and ColumnPrunerTest.
   
   ### Commits
   
    - A new ParquetRewriter is introduced to unify rewriting logic.
    - RewriteOptions is defined to provide essential settings.
    - CompressionConverter and ColumnPruner have been refactored and marked as deprecated.
   


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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1407856755

   Thank you @gszadovszky @ggershinsky @shangxinli


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377840323

   Thanks a lot @gszadovszky 


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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1067082884


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+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.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.EncryptionTestFile;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.TestFileBuilder;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;

Review Comment:
   Fixed



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382815489

   > > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively.
   > 
   > Perfect! :)
   > 
   > > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it?
   > 
   > It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`?
   
   I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation.


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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1059400234


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {

Review Comment:
   This is a good question. The code was just moved from https://github.com/apache/parquet-mr/blob/1adc22804a700d78f8480667d083e91d6147339f/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnEncryptor.java#L84. I think it should throw if the EncryptorRuntime is required but the fileEncryptor is null.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056879931


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {

Review Comment:
   Can we use i++?



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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1364593726

   If you can add more unit tests, particularly the combinations of prune, mask, trans-compression etc. 


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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1067893929


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   Yep. I meant the exception text in the line 149, it is identical to the masking check (line 142); I guess the encryption check would print "Cannot prune and encrypt same column"



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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1068159808


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   Sorry, I misunderstood your meaning. Fixed the error message. Please take a look again. Thanks!



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056877518


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName codecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName codecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.codecName = codecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getCodecName() {
+    return codecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName codecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName codecName) {
+      this.codecName = codecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      // TODO: validate any conflict setting

Review Comment:
   Can you open a Jira to track it? If you can do it in this PR, it is even better. 



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056879143


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();

Review Comment:
   There was some discussion should we carry over the old author or replace it with this rewriter author. Or we can append 'reCreatedBy' but it needs a specification (parquet-format) change. Any thoughts on it? 



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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1059401341


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {
+        this.colEncrSetup = null;
+        this.dataEncryptor =  null;
+        this.metaDataEncryptor =  null;
+
+        this.fileAAD =  null;
+        this.dataPageHeaderAAD =  null;
+        this.dataPageAAD =  null;
+        this.dictPageHeaderAAD =  null;
+        this.dictPageAAD =  null;
+      } else {
+        this.colEncrSetup = fileEncryptor.getColumnSetup(chunk.getPath(), true, columnId);
+        this.dataEncryptor = colEncrSetup.getDataEncryptor();
+        this.metaDataEncryptor = colEncrSetup.getMetaDataEncryptor();
+
+        this.fileAAD = fileEncryptor.getFileAAD();
+        this.dataPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DataPageHeader, blockId, columnId);
+        this.dataPageAAD = createAAD(colEncrSetup, ModuleType.DataPage, blockId, columnId);
+        this.dictPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DictionaryPageHeader, blockId, columnId);
+        this.dictPageAAD = createAAD(colEncrSetup, ModuleType.DictionaryPage, blockId, columnId);
+      }
+    }
+
+    private byte[] createAAD(InternalColumnEncryptionSetup colEncrSetup,
+                             ModuleType moduleType,
+                             int blockId,
+                             int columnId) {
+      if (colEncrSetup != null && colEncrSetup.isEncrypted()) {

Review Comment:
   If `EncryptorRunTime` was not created for non-encrypted columns, generating the file footer will get some error. It can be reproduced by the ColumnEncryptorTest.



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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1062185479


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName codecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName codecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.codecName = codecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getCodecName() {
+    return codecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName codecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName codecName) {
+      this.codecName = codecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      // TODO: validate any conflict setting

Review Comment:
   Fixed and removed the TODO



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName codecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName codecName,

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set

Review Comment:
   I forgot to remove this TODO.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/MaskMode.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+public enum MaskMode {
+  NULLIFY("nullify"),
+  HASH("hash"),
+  REDACT("redact");
+
+  private String mode;
+
+  MaskMode(String text) {
+    this.mode = text;
+  }
+
+  public String getMode() {
+    return this.mode;
+  }
+
+  public static MaskMode fromString(String mode) {

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();

Review Comment:
   I'd rather keep the same behavior for now and fix it in a separate JIRA.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/MaskMode.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+public enum MaskMode {
+  NULLIFY("nullify"),
+  HASH("hash"),
+  REDACT("redact");
+
+  private String mode;
+
+  MaskMode(String text) {
+    this.mode = text;

Review Comment:
   Fixed



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {

Review Comment:
   Renamed it to ColumnChunkEncryptorRunTime



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {
+        this.colEncrSetup = null;
+        this.dataEncryptor =  null;
+        this.metaDataEncryptor =  null;
+
+        this.fileAAD =  null;
+        this.dataPageHeaderAAD =  null;
+        this.dataPageAAD =  null;
+        this.dictPageHeaderAAD =  null;
+        this.dictPageAAD =  null;
+      } else {
+        this.colEncrSetup = fileEncryptor.getColumnSetup(chunk.getPath(), true, columnId);
+        this.dataEncryptor = colEncrSetup.getDataEncryptor();
+        this.metaDataEncryptor = colEncrSetup.getMetaDataEncryptor();
+
+        this.fileAAD = fileEncryptor.getFileAAD();
+        this.dataPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DataPageHeader, blockId, columnId);
+        this.dataPageAAD = createAAD(colEncrSetup, ModuleType.DataPage, blockId, columnId);
+        this.dictPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DictionaryPageHeader, blockId, columnId);
+        this.dictPageAAD = createAAD(colEncrSetup, ModuleType.DictionaryPage, blockId, columnId);
+      }
+    }
+
+    private byte[] createAAD(InternalColumnEncryptionSetup colEncrSetup,
+                             ModuleType moduleType,
+                             int blockId,
+                             int columnId) {
+      if (colEncrSetup != null && colEncrSetup.isEncrypted()) {

Review Comment:
   Fixed it by checking `colEncrSetup` only once per creation.



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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1057472437


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/MaskMode.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+public enum MaskMode {
+  NULLIFY("nullify"),
+  HASH("hash"),
+  REDACT("redact");
+
+  private String mode;
+
+  MaskMode(String text) {
+    this.mode = text;
+  }
+
+  public String getMode() {
+    return this.mode;
+  }
+
+  public static MaskMode fromString(String mode) {

Review Comment:
   I think you can use the `MaskMode.valueOf` method



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/MaskMode.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+public enum MaskMode {
+  NULLIFY("nullify"),
+  HASH("hash"),
+  REDACT("redact");
+
+  private String mode;
+
+  MaskMode(String text) {
+    this.mode = text;

Review Comment:
   string is not checked. Also, you can use the `MaskMode.valueOf` method



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {
+        this.colEncrSetup = null;
+        this.dataEncryptor =  null;
+        this.metaDataEncryptor =  null;
+
+        this.fileAAD =  null;
+        this.dataPageHeaderAAD =  null;
+        this.dataPageAAD =  null;
+        this.dictPageHeaderAAD =  null;
+        this.dictPageAAD =  null;
+      } else {
+        this.colEncrSetup = fileEncryptor.getColumnSetup(chunk.getPath(), true, columnId);
+        this.dataEncryptor = colEncrSetup.getDataEncryptor();
+        this.metaDataEncryptor = colEncrSetup.getMetaDataEncryptor();
+
+        this.fileAAD = fileEncryptor.getFileAAD();
+        this.dataPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DataPageHeader, blockId, columnId);
+        this.dataPageAAD = createAAD(colEncrSetup, ModuleType.DataPage, blockId, columnId);
+        this.dictPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DictionaryPageHeader, blockId, columnId);
+        this.dictPageAAD = createAAD(colEncrSetup, ModuleType.DictionaryPage, blockId, columnId);
+      }
+    }
+
+    private byte[] createAAD(InternalColumnEncryptionSetup colEncrSetup,
+                             ModuleType moduleType,
+                             int blockId,
+                             int columnId) {
+      if (colEncrSetup != null && colEncrSetup.isEncrypted()) {

Review Comment:
   should this be checked here, upon every call? also, what's the point of creating an EncryptorRunTime if the column is not encrypted?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {

Review Comment:
   should this object be created at all if the file encryptor is null?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {

Review Comment:
   This is a per-column-chunk collection, maybe the class name should reflect that?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set

Review Comment:
   what is left to do here?



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382752637

   
   
   
   > I think it is a great refactor. Thanks a lot for working on it, @wgtmac! In the other hand I've thought about PARQUET-2075 as a request for a new feature in `parquet-cli` that can be used to convert from one parquet file to another with specific configurations. (Later on we might extend it to allow multiple parquet files to be merged/rewritten to one specified and the tool would decide which level of deserialization/serialization is required.) I am fine with handling it in a separate jira but let's make it clear. Either create another jira for this refactor as a prerequisite of PARQUET-2075 or rephrase PARQUET-2075 and create a new for `parquet-cli`. @shangxinli, what do you think?
   
   Thanks for your review @gszadovszky 
   
   - I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively.
   - Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it?
   


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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1371885483

   > If you can add more unit tests, particularly the combinations of prune, mask, trans-compression etc, it would be better.
   
   I have added some test cases in the `ParquetRewriterTest` to mix config of prune, nullify and trans-compression. 
   
   There are two outstanding issues as below:
   - Masking and encrypting same column is not supported yet.
   - Fix `createdBy` of rewritten file to keep both original info and new rewriter info.
   
   I am inclined to fix them in separate JIRAs because this patch is simply a refactoring patch to unify different rewriters without changing any behavior, and it is large enough.
   
   @ggershinsky @shangxinli Please review again when you have time. Thanks!


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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1407653245

   Thank you, @wgtmac for working on this! It looks good to me.
   Since there were other reviewers, I would wait a bit to give a chance to them for add feedback. I'll push it after a couple of days. (Feel free to ping me if I forget.)


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   No, pruned column does not exist in the rewritten file. Then it does not make sense to encrypt the missing column any more.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   Pruned column does not exist in the rewritten file. Then it does not make sense to encrypt the missing column any more.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056882191


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);

Review Comment:
   This line seems a bug(even in the original code). We should remove by index instead of by name. Something like this: currentPath.remove(currentPath.size() - 1)



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056881216


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());

Review Comment:
   Can we apply this.encryptMode to this check also? 



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1407613955

   Gentle ping. @gszadovszky @ggershinsky @shangxinli
   
   Any chance to take another look?


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1383101451

   > > I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation.
   > 
   > I do not have a strong opinion for `'\n'` only that we need a character that probably won't be used by any systems writing parquet files.
   
   As we are discussing a new entry (`original.created.by`) to the key value metadata, I need to raise two related issues once we have supported rewriting (merging) several files into one:
   - We need to merge `original.created.by` from all input files, making it difficult to tell which created_by comes from which input file. Therefore, `original.created.by` should be dropped in this case.
   - Is there any key value metadata that will conflict from different input files and should be dealt with by the rewriter? For now we simply keep all the old key value metadata from the old file.
   
   @gszadovszky @ggershinsky @shangxinli Thoughts?
   
   If this behavior requires further discussion, I'd suggest to keep the current state of `created_by` unchanged in this pull request which is large enough. All rewriters (ColumnPruner, CompressionConverter, ColumnMasker, and ColumnEncrypter) have dropped original `created_by` and store the current writer version to the footer.
   
   


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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382840526

   > I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation.
   
   I do not have a strong opinion for `'\n'` only that we need a character that probably won't be used by any systems writing parquet files.


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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1064348254


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   Cannot prune and encrypt same column?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+      }
+
+      // TODO: support masking and encrypting same columns
+      if (maskColumns != null && encryptColumns != null) {
+        for (String encryptColumn : encryptColumns) {
+          Preconditions.checkArgument(!maskColumns.containsKey(encryptColumn),
+                  "Cannot encrypt and mask same column");
+        }
+      }
+
+      if (encryptColumns != null && !encryptColumns.isEmpty()) {
+        Preconditions.checkArgument(fileEncryptionProperties != null,

Review Comment:
   what if `fileEncryptionProperties` was set, but the `encryptColumns` is null or `encryptColumns.isEmpty()`?



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+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.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.EncryptionTestFile;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.TestFileBuilder;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.*;
+import static org.junit.Assert.*;
+
+public class ParquetRewriterTest {
+
+  private final int numRecord = 100000;
+  private Configuration conf = new Configuration();
+  private EncryptionTestFile inputFile = null;
+  private String outputFile = null;
+  private ParquetRewriter rewriter = null;
+
+  @Test
+  public void testPruneSingleColumnAndTranslateCodec() throws Exception {

Review Comment:
   do we also have an encryption rewrite unitest?



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+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.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.EncryptionTestFile;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.TestFileBuilder;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;

Review Comment:
   please don't use * (also in lines 57, 58)



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1383152341

   I agree that merging the key-value metadata is not an easy question. Let's discuss it separately as it is not related to this PR.
   
   I also agree to store the current writer (parquet-mr) in `created_by` in case of rewriting.  It is not easy to decide what would be the proper solution anyway. `created_by` is usually used for handling potential erroneous writes. Let's say there was an issue in parquet-mr at the version 1.2.3 that written a specific encoding of integers wrongly (not according to spec). What if we rewrite the file but do not re-encode the pages? Can we still handle the original issue? What if the rewriter re-encodes the related pages?
   Let's store the original writer in `original.created.by` for now. Let's discuss this topic separately however, I am not sure if we can find a proper solution.


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


[GitHub] [parquet-mr] gszadovszky merged pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky merged PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] ggershinsky commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1364908453

   > I just left some comments initially. I will spend more time on it.
   > 
   > @ggershinsky If you have time, can you have a look too?
   
   sure, will do


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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1059392324


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();

Review Comment:
   IMHO, reusing the old `createdBy` is a bad idea which makes it difficult to reason about the origin of the file. What about concatenating the old `createdBy` with the writer version of `ParquetRewriter` so we can keep both old and new creator in the same field?



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056877460


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName codecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName codecName,

Review Comment:
   Maybe use the variable 'newCodecName'? 



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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1059612222


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {
+        this.colEncrSetup = null;
+        this.dataEncryptor =  null;
+        this.metaDataEncryptor =  null;
+
+        this.fileAAD =  null;
+        this.dataPageHeaderAAD =  null;
+        this.dataPageAAD =  null;
+        this.dictPageHeaderAAD =  null;
+        this.dictPageAAD =  null;
+      } else {
+        this.colEncrSetup = fileEncryptor.getColumnSetup(chunk.getPath(), true, columnId);
+        this.dataEncryptor = colEncrSetup.getDataEncryptor();
+        this.metaDataEncryptor = colEncrSetup.getMetaDataEncryptor();
+
+        this.fileAAD = fileEncryptor.getFileAAD();
+        this.dataPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DataPageHeader, blockId, columnId);
+        this.dataPageAAD = createAAD(colEncrSetup, ModuleType.DataPage, blockId, columnId);
+        this.dictPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DictionaryPageHeader, blockId, columnId);
+        this.dictPageAAD = createAAD(colEncrSetup, ModuleType.DictionaryPage, blockId, columnId);
+      }
+    }
+
+    private byte[] createAAD(InternalColumnEncryptionSetup colEncrSetup,
+                             ModuleType moduleType,
+                             int blockId,
+                             int columnId) {
+      if (colEncrSetup != null && colEncrSetup.isEncrypted()) {

Review Comment:
   not sure I understand. The EncryptorRunTime object is created per column, so no need to run this check for every module AAD; this boolean can be built in the EncryptorRunTime constructor. Also, it would be more logical to use EncryptorRunTime objects only for encrypted columns. 



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1056879313


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);

Review Comment:
   Should we move the line 120 to be inside the getPath()?



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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1364592919

   I just left some comments initially. I will spend more time on it. 
   
   @ggershinsky If you have time, can you have a look too? 


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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1383160117

   > I agree that merging the key-value metadata is not an easy question. Let's discuss it separately as it is not related to this PR.
   > 
   > I also agree to store the current writer (parquet-mr) in `created_by` in case of rewriting. It is not easy to decide what would be the proper solution anyway. `created_by` is usually used for handling potential erroneous writes. Let's say there was an issue in parquet-mr at the version 1.2.3 that written a specific encoding of integers wrongly (not according to spec). What if we rewrite the file but do not re-encode the pages? Can we still handle the original issue? What if the rewriter re-encodes the related pages? Let's store the original writer in `original.created.by` for now. Let's discuss this topic separately however, I am not sure if we can find a proper solution.
   
   I agree. Now I have updated this PR to preserve the old writer version into `original.created.by` and added a test to make sure it is preserved. Please take a look. Thanks!


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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1067082250


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+      }
+
+      // TODO: support masking and encrypting same columns
+      if (maskColumns != null && encryptColumns != null) {
+        for (String encryptColumn : encryptColumns) {
+          Preconditions.checkArgument(!maskColumns.containsKey(encryptColumn),
+                  "Cannot encrypt and mask same column");
+        }
+      }
+
+      if (encryptColumns != null && !encryptColumns.isEmpty()) {
+        Preconditions.checkArgument(fileEncryptionProperties != null,

Review Comment:
   I have added checking to disable this case.



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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1070274495


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();

Review Comment:
   Since we have only one field for _created by_ and it's content is more or less specified, we should write the current version of parquet-mr there. It is also a good idea to keep the original _created by_ value, though. What do you think about adding it to the `key_value_metadata` with a specific key? Even though this field would not be specified at least we won't lose the info.



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382754916

   > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively.
   
   Perfect! :)
   
   > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it?
   
   It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`?
   


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


[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1351676813

   Can you please take a look when you have time? @shangxinli @gszadovszky @ggershinsky 


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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1067083773


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+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.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.EncryptionTestFile;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.TestFileBuilder;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.*;
+import static org.junit.Assert.*;
+
+public class ParquetRewriterTest {
+
+  private final int numRecord = 100000;
+  private Configuration conf = new Configuration();
+  private EncryptionTestFile inputFile = null;
+  private String outputFile = null;
+  private ParquetRewriter rewriter = null;
+
+  @Test
+  public void testPruneSingleColumnAndTranslateCodec() throws Exception {

Review Comment:
   Added a test case to prune, transcode and encrypt columns.



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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List<String> pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map<String, MaskMode> maskColumns;
+  final List<String> encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+                         Path inputFile,
+                         Path outputFile,
+                         List<String> pruneColumns,
+                         CompressionCodecName newCodecName,
+                         Map<String, MaskMode> maskColumns,
+                         List<String> encryptColumns,
+                         FileEncryptionProperties fileEncryptionProperties) {
+    this.conf = conf;
+    this.inputFile = inputFile;
+    this.outputFile = outputFile;
+    this.pruneColumns = pruneColumns;
+    this.newCodecName = newCodecName;
+    this.maskColumns = maskColumns;
+    this.encryptColumns = encryptColumns;
+    this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public Path getInputFile() {
+    return inputFile;
+  }
+
+  public Path getOutputFile() {
+    return outputFile;
+  }
+
+  public List<String> getPruneColumns() {
+    return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+    return newCodecName;
+  }
+
+  public Map<String, MaskMode> getMaskColumns() {
+    return maskColumns;
+  }
+
+  public List<String> getEncryptColumns() {
+    return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+    return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+    private Configuration conf;
+    private Path inputFile;
+    private Path outputFile;
+    private List<String> pruneColumns;
+    private CompressionCodecName newCodecName;
+    private Map<String, MaskMode> maskColumns;
+    private List<String> encryptColumns;
+    private FileEncryptionProperties fileEncryptionProperties;
+
+    public Builder(Configuration conf, Path inputFile, Path outputFile) {
+      this.conf = conf;
+      this.inputFile = inputFile;
+      this.outputFile = outputFile;
+    }
+
+    public Builder prune(List<String> columns) {
+      this.pruneColumns = columns;
+      return this;
+    }
+
+    public Builder transform(CompressionCodecName newCodecName) {
+      this.newCodecName = newCodecName;
+      return this;
+    }
+
+    public Builder mask(Map<String, MaskMode> maskColumns) {
+      this.maskColumns = maskColumns;
+      return this;
+    }
+
+    public Builder encrypt(List<String> encryptColumns) {
+      this.encryptColumns = encryptColumns;
+      return this;
+    }
+
+    public Builder encryptionProperties(FileEncryptionProperties fileEncryptionProperties) {
+      this.fileEncryptionProperties = fileEncryptionProperties;
+      return this;
+    }
+
+    public RewriteOptions build() {
+      Preconditions.checkArgument(inputFile != null, "Input file is required");
+      Preconditions.checkArgument(outputFile != null, "Output file is required");
+
+      if (pruneColumns != null) {
+        if (maskColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+                    "Cannot prune and mask same column");
+          }
+        }
+
+        if (encryptColumns != null) {
+          for (String pruneColumn : pruneColumns) {
+            Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+                    "Cannot prune and mask same column");

Review Comment:
   Yes, pruned column does not exist in the rewritten file. Then it does not make sense to encrypt the missing column any more.



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377706622

   > @gszadovszky I Just want to check if you have time to have a look. @wgtmac just be nice to take over the work that we discussed earlier to have an aggregated rewriter.
   
   @shangxinli, I'll try to take a look this week.


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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1376754942

   @gszadovszky I Just want to check if you have time to have a look. @wgtmac Just be nice to take over the work that we discussed earlier to have an aggregated rewriter. 


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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation

Posted by "shangxinli (via GitHub)" <gi...@apache.org>.
shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1407713001

   Thanks @wgtmac for working on this and Thanks @gszadovszky and @ggershinsky for reviewing it! I am a little late for the comments discussion but I see we are in the right direction. Let's address it in a separate discussion. If it turns out that changing the parquet-format is the right way to solve it, we can make the proposal and I can help for the approval. 
   
   My comments are all addressed. I don't have further comments. 


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1059612222


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -0,0 +1,733 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriteStore;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.crypto.AesCipher;
+import org.apache.parquet.crypto.InternalColumnEncryptionSetup;
+import org.apache.parquet.crypto.InternalFileEncryptor;
+import org.apache.parquet.format.BlockCipher;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.DictionaryPageHeader;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.ColumnChunkPageWriteStore;
+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.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH;
+import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;
+
+public class ParquetRewriter implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class);
+  private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2;
+  private final byte[] pageBuffer = new byte[pageBufferSize];
+  private TransParquetFileReader reader;
+  private ParquetFileWriter writer;
+  private ParquetMetadata meta;
+  private MessageType schema;
+  private String createdBy;
+  private CompressionCodecName codecName = null;
+  private List<String> pruneColumns = null;
+  private Map<ColumnPath, MaskMode> maskColumns = null;
+  private Set<ColumnPath> encryptColumns = null;
+  private boolean encryptMode = false;
+
+  public ParquetRewriter(RewriteOptions options) throws IOException {
+    Path inPath = options.getInputFile();
+    Path outPath = options.getOutputFile();
+    Configuration conf = options.getConf();
+
+    // TODO: set more member variables
+    codecName = options.getCodecName();
+    pruneColumns = options.getPruneColumns();
+
+    // Get file metadata and full schema from the input file
+    meta = ParquetFileReader.readFooter(conf, inPath, NO_FILTER);
+    schema = meta.getFileMetaData().getSchema();
+    createdBy = meta.getFileMetaData().getCreatedBy();
+
+    // Prune columns if specified
+    if (pruneColumns != null && !pruneColumns.isEmpty()) {
+      List<String> paths = new ArrayList<>();
+      getPaths(schema, paths, null);
+      for (String col : pruneColumns) {
+        if (!paths.contains(col)) {
+          LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, inPath.getName());
+        }
+      }
+
+      Set<ColumnPath> prunePaths = convertToColumnPaths(pruneColumns);
+      schema = pruneColumnsInSchema(schema, prunePaths);
+    }
+
+    if (options.getMaskColumns() != null) {
+      this.maskColumns = new HashMap<>();
+      for (Map.Entry<String, MaskMode> col : options.getMaskColumns().entrySet()) {
+        maskColumns.put(ColumnPath.fromDotString(col.getKey()), col.getValue());
+      }
+    }
+
+    if (options.getEncryptColumns() != null && options.getFileEncryptionProperties() != null) {
+      this.encryptColumns = convertToColumnPaths(options.getEncryptColumns());
+      this.encryptMode = true;
+      // TODO: make sure options.getFileEncryptionProperties() is set
+    }
+
+    reader = new TransParquetFileReader(
+            HadoopInputFile.fromPath(inPath, conf), HadoopReadOptions.builder(conf).build());
+
+    ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE;
+    writer = new ParquetFileWriter(HadoopOutputFile.fromPath(outPath, conf), schema, writerMode,
+            DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH,
+            DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED,
+            options.getFileEncryptionProperties());
+    writer.start();
+  }
+
+  // Ctor for legacy CompressionConverter and ColumnMasker
+  public ParquetRewriter(TransParquetFileReader reader,
+                         ParquetFileWriter writer,
+                         ParquetMetadata meta,
+                         MessageType schema,
+                         String createdBy,
+                         CompressionCodecName codecName,
+                         List<String> maskColumns,
+                         MaskMode maskMode) {
+    this.reader = reader;
+    this.writer = writer;
+    this.meta = meta;
+    this.schema = schema;
+    this.createdBy = createdBy == null ? meta.getFileMetaData().getCreatedBy() : createdBy;
+    this.codecName = codecName;
+    if (maskColumns != null && maskMode != null) {
+      this.maskColumns = new HashMap<>();
+      for (String col : maskColumns) {
+        this.maskColumns.put(ColumnPath.fromDotString(col), maskMode);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.end(meta.getFileMetaData().getKeyValueMetaData());
+  }
+
+  public void processBlocks() throws IOException {
+    PageReadStore store = reader.readNextRowGroup();
+    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, createdBy);
+    Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+            Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+
+    int blockId = 0;
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+
+      BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      List<ColumnChunkMetaData> columnsInOrder = blockMetaData.getColumns();
+
+      for (int i = 0, columnId = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+
+        // This column has been pruned.
+        if (descriptor == null) {
+          continue;
+        }
+
+        // If a column is encrypted, we simply throw exception.
+        // Later we can add a feature to trans-encrypt it with different keys
+        if (chunk.isEncrypted()) {
+          throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted");
+        }
+
+        reader.setStreamPosition(chunk.getStartingPos());
+        CompressionCodecName newCodecName = codecName == null ? chunk.getCodec() : codecName;
+        EncryptorRunTime encryptorRunTime = null;
+        if (this.encryptMode) {
+          encryptorRunTime = new EncryptorRunTime(writer.getEncryptor(), chunk, blockId, columnId);
+        }
+        boolean encryptColumn = encryptColumns != null && encryptColumns.contains(chunk.getPath());
+
+        if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) {
+          // Mask column and compress it again.
+          MaskMode maskMode = maskColumns.get(chunk.getPath());
+          if (maskMode.equals(MaskMode.NULLIFY)) {
+            Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+            if (repetition.equals(Type.Repetition.REQUIRED)) {
+              throw new IOException(
+                      "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+            }
+            nullifyColumn(descriptor, chunk, crStore, writer, schema, newCodecName, encryptorRunTime, encryptColumn);
+          } else {
+            throw new UnsupportedOperationException("Only nullify is supported for now");
+          }
+        } else if (encryptMode || codecName != null) {
+          // Translate compression and/or encryption
+          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          processChunk(chunk, newCodecName, encryptorRunTime, encryptColumn);
+          writer.endColumn();
+        } else {
+          // Nothing changed, simply copy the binary data.
+          BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+          ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+          OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+          writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+        }
+
+        columnId++;
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockId++;
+    }
+  }
+
+  private void processChunk(ColumnChunkMetaData chunk,
+                            CompressionCodecName newCodecName,
+                            EncryptorRunTime encryptorRunTime,
+                            boolean encryptColumn) throws IOException {
+    CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0);
+    CompressionCodecFactory.BytesInputDecompressor decompressor = null;
+    CompressionCodecFactory.BytesInputCompressor compressor = null;
+    if (!newCodecName.equals(chunk.getCodec())) {
+      // Re-compress only if a different codec has been specified
+      decompressor = codecFactory.getDecompressor(chunk.getCodec());
+      compressor = codecFactory.getCompressor(newCodecName);
+    }
+
+    // EncryptorRunTime is only provided when encryption is required
+    BlockCipher.Encryptor metaEncryptor = null;
+    BlockCipher.Encryptor dataEncryptor = null;
+    byte[] dictPageAAD = null;
+    byte[] dataPageAAD = null;
+    byte[] dictPageHeaderAAD = null;
+    byte[] dataPageHeaderAAD = null;
+    if (encryptorRunTime != null) {
+      metaEncryptor = encryptorRunTime.getMetaDataEncryptor();
+      dataEncryptor = encryptorRunTime.getDataEncryptor();
+      dictPageAAD = encryptorRunTime.getDictPageAAD();
+      dataPageAAD = encryptorRunTime.getDataPageAAD();
+      dictPageHeaderAAD = encryptorRunTime.getDictPageHeaderAAD();
+      dataPageHeaderAAD = encryptorRunTime.getDataPageHeaderAAD();
+    }
+
+    ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+    OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+
+    reader.setStreamPosition(chunk.getStartingPos());
+    DictionaryPage dictionaryPage = null;
+    long readValues = 0;
+    Statistics statistics = null;
+    ParquetMetadataConverter converter = new ParquetMetadataConverter();
+    int pageOrdinal = 0;
+    long totalChunkValues = chunk.getValueCount();
+    while (readValues < totalChunkValues) {
+      PageHeader pageHeader = reader.readPageHeader();
+      int compressedPageSize = pageHeader.getCompressed_page_size();
+      byte[] pageLoad;
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          if (dictionaryPage != null) {
+            throw new IOException("has more than one dictionary page in column chunk");
+          }
+          //No quickUpdatePageAAD needed for dictionary page
+          DictionaryPageHeader dictPageHeader = pageHeader.dictionary_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dictPageAAD);
+          writer.writeDictionaryPage(new DictionaryPage(BytesInput.from(pageLoad),
+                  pageHeader.getUncompressed_page_size(),
+                  dictPageHeader.getNum_values(),
+                  converter.getEncoding(dictPageHeader.getEncoding())),
+                  metaEncryptor,
+                  dictPageHeaderAAD);
+          break;
+        case DATA_PAGE:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeader headerV1 = pageHeader.data_page_header;
+          pageLoad = processPageLoad(reader,
+                  true,
+                  compressor,
+                  decompressor,
+                  pageHeader.getCompressed_page_size(),
+                  pageHeader.getUncompressed_page_size(),
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV1.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV1.getNum_values();
+          if (offsetIndex != null) {
+            long rowCount = 1 + offsetIndex.getLastRowIndex(
+                    pageOrdinal, totalChunkValues) - offsetIndex.getFirstRowIndex(pageOrdinal);
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    toIntWithCheck(rowCount),
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          } else {
+            writer.writeDataPage(toIntWithCheck(headerV1.getNum_values()),
+                    pageHeader.getUncompressed_page_size(),
+                    BytesInput.from(pageLoad),
+                    statistics,
+                    converter.getEncoding(headerV1.getRepetition_level_encoding()),
+                    converter.getEncoding(headerV1.getDefinition_level_encoding()),
+                    converter.getEncoding(headerV1.getEncoding()),
+                    metaEncryptor,
+                    dataPageHeaderAAD);
+          }
+          pageOrdinal++;
+          break;
+        case DATA_PAGE_V2:
+          if (encryptColumn) {
+            AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);
+            AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
+          }
+          DataPageHeaderV2 headerV2 = pageHeader.data_page_header_v2;
+          int rlLength = headerV2.getRepetition_levels_byte_length();
+          BytesInput rlLevels = readBlockAllocate(rlLength, reader);
+          int dlLength = headerV2.getDefinition_levels_byte_length();
+          BytesInput dlLevels = readBlockAllocate(dlLength, reader);
+          int payLoadLength = pageHeader.getCompressed_page_size() - rlLength - dlLength;
+          int rawDataLength = pageHeader.getUncompressed_page_size() - rlLength - dlLength;
+          pageLoad = processPageLoad(
+                  reader,
+                  headerV2.is_compressed,
+                  compressor,
+                  decompressor,
+                  payLoadLength,
+                  rawDataLength,
+                  encryptColumn,
+                  dataEncryptor,
+                  dataPageAAD);
+          statistics = convertStatistics(
+                  createdBy, chunk.getPrimitiveType(), headerV2.getStatistics(), columnIndex, pageOrdinal, converter);
+          readValues += headerV2.getNum_values();
+          writer.writeDataPageV2(headerV2.getNum_rows(),
+                  headerV2.getNum_nulls(),
+                  headerV2.getNum_values(),
+                  rlLevels,
+                  dlLevels,
+                  converter.getEncoding(headerV2.getEncoding()),
+                  BytesInput.from(pageLoad),
+                  rawDataLength,
+                  statistics);
+          pageOrdinal++;
+          break;
+        default:
+          LOG.debug("skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize);
+          break;
+      }
+    }
+  }
+
+  private Statistics convertStatistics(String createdBy,
+                                       PrimitiveType type,
+                                       org.apache.parquet.format.Statistics pageStatistics,
+                                       ColumnIndex columnIndex,
+                                       int pageIndex,
+                                       ParquetMetadataConverter converter) throws IOException {
+    if (columnIndex != null) {
+      if (columnIndex.getNullPages() == null) {
+        throw new IOException("columnIndex has null variable 'nullPages' which indicates corrupted data for type: " +
+                type.getName());
+      }
+      if (pageIndex > columnIndex.getNullPages().size()) {
+        throw new IOException("There are more pages " + pageIndex + " found in the column than in the columnIndex " +
+                columnIndex.getNullPages().size());
+      }
+      org.apache.parquet.column.statistics.Statistics.Builder statsBuilder =
+              org.apache.parquet.column.statistics.Statistics.getBuilderForReading(type);
+      statsBuilder.withNumNulls(columnIndex.getNullCounts().get(pageIndex));
+
+      if (!columnIndex.getNullPages().get(pageIndex)) {
+        statsBuilder.withMin(columnIndex.getMinValues().get(pageIndex).array().clone());
+        statsBuilder.withMax(columnIndex.getMaxValues().get(pageIndex).array().clone());
+      }
+      return statsBuilder.build();
+    } else if (pageStatistics != null) {
+      return converter.fromParquetStatistics(createdBy, pageStatistics, type);
+    } else {
+      return null;
+    }
+  }
+
+  private byte[] processPageLoad(TransParquetFileReader reader,
+                                 boolean isCompressed,
+                                 CompressionCodecFactory.BytesInputCompressor compressor,
+                                 CompressionCodecFactory.BytesInputDecompressor decompressor,
+                                 int payloadLength,
+                                 int rawDataLength,
+                                 boolean encrypt,
+                                 BlockCipher.Encryptor dataEncryptor,
+                                 byte[] AAD) throws IOException {
+    BytesInput data = readBlock(payloadLength, reader);
+
+    // recompress page load
+    if (compressor != null) {
+      if (isCompressed) {
+        data = decompressor.decompress(data, rawDataLength);
+      }
+      data = compressor.compress(data);
+    }
+
+    if (!encrypt) {
+      return data.toByteArray();
+    }
+
+    // encrypt page load
+    return dataEncryptor.encrypt(data.toByteArray(), AAD);
+  }
+
+  public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data;
+    if (length > pageBufferSize) {
+      data = new byte[length];
+    } else {
+      data = pageBuffer;
+    }
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  public BytesInput readBlockAllocate(int length, TransParquetFileReader reader) throws IOException {
+    byte[] data = new byte[length];
+    reader.blockRead(data, 0, length);
+    return BytesInput.from(data, 0, length);
+  }
+
+  private int toIntWithCheck(long size) {
+    if ((int)size != size) {
+      throw new ParquetEncodingException("size is bigger than " + Integer.MAX_VALUE + " bytes: " + size);
+    }
+    return (int)size;
+  }
+
+  // We have to rewrite getPaths because MessageType only get level 0 paths
+  private void getPaths(GroupType schema, List<String> paths, String parent) {
+    List<Type> fields = schema.getFields();
+    String prefix = (parent == null) ? "" : parent + ".";
+    for (Type field : fields) {
+      paths.add(prefix + field.getName());
+      if (field instanceof GroupType) {
+        getPaths(field.asGroupType(), paths, prefix + field.getName());
+      }
+    }
+  }
+
+  private MessageType pruneColumnsInSchema(MessageType schema, Set<ColumnPath> prunePaths) {
+    List<Type> fields = schema.getFields();
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths);
+    MessageType newSchema = new MessageType(schema.getName(), prunedFields);
+    return newSchema;
+  }
+
+  private List<Type> pruneColumnsInFields(List<Type> fields, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    List<Type> prunedFields = new ArrayList<>();
+    for (Type childField : fields) {
+      Type prunedChildField = pruneColumnsInField(childField, currentPath, prunePaths);
+      if (prunedChildField != null) {
+        prunedFields.add(prunedChildField);
+      }
+    }
+    return prunedFields;
+  }
+
+  private Type pruneColumnsInField(Type field, List<String> currentPath, Set<ColumnPath> prunePaths) {
+    String fieldName = field.getName();
+    currentPath.add(fieldName);
+    ColumnPath path = ColumnPath.get(currentPath.toArray(new String[0]));
+    Type prunedField = null;
+    if (!prunePaths.contains(path)) {
+      if (field.isPrimitive()) {
+        prunedField = field;
+      } else {
+        List<Type> childFields = ((GroupType) field).getFields();
+        List<Type> prunedFields = pruneColumnsInFields(childFields, currentPath, prunePaths);
+        if (prunedFields.size() > 0) {
+          prunedField = ((GroupType) field).withNewFields(prunedFields);
+        }
+      }
+    }
+
+    currentPath.remove(fieldName);
+    return prunedField;
+  }
+
+  private Set<ColumnPath> convertToColumnPaths(List<String> cols) {
+    Set<ColumnPath> prunePaths = new HashSet<>();
+    for (String col : cols) {
+      prunePaths.add(ColumnPath.fromDotString(col));
+    }
+    return prunePaths;
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor,
+                             ColumnChunkMetaData chunk,
+                             ColumnReadStoreImpl crStore,
+                             ParquetFileWriter writer,
+                             MessageType schema,
+                             CompressionCodecName newCodecName,
+                             EncryptorRunTime encryptorRunTime,
+                             boolean encryptColumn) throws IOException {
+    // TODO: support encryption
+    if (encryptorRunTime != null) {
+      throw new RuntimeException("Nullifying and encrypting column is not implemented yet");
+    }
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ?
+            ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+            .withWriterVersion(writerVersion)
+            .build();
+    CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold());
+    CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(newCodecName);
+
+    // Create new schema that only has the current column
+    MessageType newSchema = newSchema(schema, descriptor);
+    ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore(
+            compressor, newSchema, props.getAllocator(), props.getColumnIndexTruncateLength());
+    ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore);
+    ColumnWriter cWriter = cStore.getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " +
+                  chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level,
+        // instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+      cStore.endRecord();
+    }
+
+    cStore.flush();
+    cPageStore.flushToFileWriter(writer);
+
+    cStore.close();
+    cWriter.close();
+  }
+
+  private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) {
+    String[] path = descriptor.getPath();
+    Type type = schema.getType(path);
+    if (path.length == 1) {
+      return new MessageType(schema.getName(), type);
+    }
+
+    for (Type field : schema.getFields()) {
+      if (!field.isPrimitive()) {
+        Type newType = extractField(field.asGroupType(), type);
+        if (newType != null) {
+          return new MessageType(schema.getName(), newType);
+        }
+      }
+    }
+
+    // We should never hit this because 'type' is returned by schema.getType().
+    throw new RuntimeException("No field is found");
+  }
+
+  private Type extractField(GroupType candidate, Type targetField) {
+    if (targetField.equals(candidate)) {
+      return targetField;
+    }
+
+    // In case 'type' is a descendants of candidate
+    for (Type field : candidate.asGroupType().getFields()) {
+      if (field.isPrimitive()) {
+        if (field.equals(targetField)) {
+          return new GroupType(candidate.getRepetition(), candidate.getName(), targetField);
+        }
+      } else {
+        Type tempField = extractField(field.asGroupType(), targetField);
+        if (tempField != null) {
+          return tempField;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static final class DummyGroupConverter extends GroupConverter {
+    @Override public void start() {}
+    @Override public void end() {}
+    @Override public Converter getConverter(int fieldIndex) { return new DummyConverter(); }
+  }
+
+  private static final class DummyConverter extends PrimitiveConverter {
+    @Override public GroupConverter asGroupConverter() { return new DummyGroupConverter(); }
+  }
+
+  private static class EncryptorRunTime {
+    private final InternalColumnEncryptionSetup colEncrSetup;
+    private final BlockCipher.Encryptor dataEncryptor;
+    private final BlockCipher.Encryptor metaDataEncryptor;
+    private final byte[] fileAAD ;
+
+    private byte[] dataPageHeaderAAD;
+    private byte[] dataPageAAD;
+    private byte[] dictPageHeaderAAD;
+    private byte[] dictPageAAD;
+
+    public EncryptorRunTime(InternalFileEncryptor fileEncryptor,
+                            ColumnChunkMetaData chunk,
+                            int blockId,
+                            int columnId) throws IOException  {
+      if (fileEncryptor == null) {
+        this.colEncrSetup = null;
+        this.dataEncryptor =  null;
+        this.metaDataEncryptor =  null;
+
+        this.fileAAD =  null;
+        this.dataPageHeaderAAD =  null;
+        this.dataPageAAD =  null;
+        this.dictPageHeaderAAD =  null;
+        this.dictPageAAD =  null;
+      } else {
+        this.colEncrSetup = fileEncryptor.getColumnSetup(chunk.getPath(), true, columnId);
+        this.dataEncryptor = colEncrSetup.getDataEncryptor();
+        this.metaDataEncryptor = colEncrSetup.getMetaDataEncryptor();
+
+        this.fileAAD = fileEncryptor.getFileAAD();
+        this.dataPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DataPageHeader, blockId, columnId);
+        this.dataPageAAD = createAAD(colEncrSetup, ModuleType.DataPage, blockId, columnId);
+        this.dictPageHeaderAAD = createAAD(colEncrSetup, ModuleType.DictionaryPageHeader, blockId, columnId);
+        this.dictPageAAD = createAAD(colEncrSetup, ModuleType.DictionaryPage, blockId, columnId);
+      }
+    }
+
+    private byte[] createAAD(InternalColumnEncryptionSetup colEncrSetup,
+                             ModuleType moduleType,
+                             int blockId,
+                             int columnId) {
+      if (colEncrSetup != null && colEncrSetup.isEncrypted()) {

Review Comment:
   not sureI understand. The EncryptorRunTime object is created per column, so no need to run this check for every module aad; this boolean can be built in the EncryptorRunTime constructor. Also, it would be more logical to use EncryptorRunTime objects only for encrypted columns. 



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