[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676916#comment-17676916 ] ASF GitHub Bot commented on PARQUET-2075: - 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676912#comment-17676912 ] ASF GitHub Bot commented on PARQUET-2075: - 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'`? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676909#comment-17676909 ] ASF GitHub Bot commented on PARQUET-2075: - 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? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676905#comment-17676905 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676033#comment-17676033 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17675904#comment-17675904 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17675613#comment-17675613 ] ASF GitHub Bot commented on PARQUET-2075: - 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 = 10; + 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17675611#comment-17675611 ] ASF GitHub Bot commented on PARQUET-2075: - 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 > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17675610#comment-17675610 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17656966#comment-17656966 ] ASF GitHub Bot commented on PARQUET-2075: - shangxinli commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377840323 Thanks a lot @gszadovszky > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17656749#comment-17656749 ] ASF GitHub Bot commented on PARQUET-2075: - 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17656692#comment-17656692 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17656693#comment-17656693 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17656426#comment-17656426 ] ASF GitHub Bot commented on PARQUET-2075: - 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17655956#comment-17655956 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName newCodecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName newCodecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getNewCodecName() { +return newCodecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List 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 pruneColumns; +private CompressionCodecName newCodecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName newCodecName) { + this.newCodecName = newCodecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17654842#comment-17654842 ] ASF GitHub Bot commented on PARQUET-2075: - 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! > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17654840#comment-17654840 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName codecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName codecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getCodecName() { +return codecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List getEncryptColumns() { +return encryptColumns; + } + + public FileEncryptionProperties getFileEncryptionProperties() { +return fileEncryptionProperties; + } + + public static class Builder { +private Configuration conf; +private Path inputFile; +private Path outputFile; +private List pruneColumns; +private CompressionCodecName codecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName codecName) { + this.codecName = codecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653279#comment-17653279 ] ASF GitHub Bot commented on PARQUET-2075: - ggershinsky commented on code in PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r105961 ## 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653278#comment-17653278 ] ASF GitHub Bot commented on PARQUET-2075: - ggershinsky commented on code in PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r105961 ## 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653139#comment-17653139 ] ASF GitHub Bot commented on PARQUET-2075: - 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 =
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653138#comment-17653138 ] ASF GitHub Bot commented on PARQUET-2075: - 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 =
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653135#comment-17653135 ] ASF GitHub Bot commented on PARQUET-2075: - 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 =
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17652175#comment-17652175 ] ASF GitHub Bot commented on PARQUET-2075: - 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;
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651930#comment-17651930 ] ASF GitHub Bot commented on PARQUET-2075: - 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 > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651841#comment-17651841 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651840#comment-17651840 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651839#comment-17651839 ] ASF GitHub Bot commented on PARQUET-2075: - 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651838#comment-17651838 ] ASF GitHub Bot commented on PARQUET-2075: - 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? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651837#comment-17651837 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651836#comment-17651836 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651835#comment-17651835 ] ASF GitHub Bot commented on PARQUET-2075: - 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
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651834#comment-17651834 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName codecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName codecName, + Map maskColumns, + List 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 getPruneColumns() { +return pruneColumns; + } + + public CompressionCodecName getCodecName() { +return codecName; + } + + public Map getMaskColumns() { +return maskColumns; + } + + public List getEncryptColumns() { +return encryptColumns; + } + + public FileEncryptionProperties getFileEncryptionProperties() { +return fileEncryptionProperties; + } + + public static class Builder { +private Configuration conf; +private Path inputFile; +private Path outputFile; +private List pruneColumns; +private CompressionCodecName codecName; +private Map maskColumns; +private List 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 columns) { + this.pruneColumns = columns; + return this; +} + +public Builder transform(CompressionCodecName codecName) { + this.codecName = codecName; + return this; +} + +public Builder mask(Map maskColumns) { + this.maskColumns = maskColumns; + return this; +} + +public Builder encrypt(List 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651833#comment-17651833 ] ASF GitHub Bot commented on PARQUET-2075: - 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 pruneColumns; + final CompressionCodecName codecName; + final Map maskColumns; + final List encryptColumns; + final FileEncryptionProperties fileEncryptionProperties; + + private RewriteOptions(Configuration conf, + Path inputFile, + Path outputFile, + List pruneColumns, + CompressionCodecName codecName, Review Comment: Maybe use the variable 'newCodecName'? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17647563#comment-17647563 ] ASF GitHub Bot commented on PARQUET-2075: - 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 > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17646980#comment-17646980 ] ASF GitHub Bot commented on PARQUET-2075: - 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. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17645349#comment-17645349 ] Gang Wu commented on PARQUET-2075: -- As discussed offline, I will work on it. So I just changed the assignee to me. [~sha...@uber.com] > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)