[ 
https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 = 100000;
+  private Configuration conf = new Configuration();
+  private EncryptionTestFile inputFile = null;
+  private String outputFile = null;
+  private ParquetRewriter rewriter = null;
+
+  @Test
+  public void testPruneSingleColumnAndTranslateCodec() throws Exception {

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





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

Reply via email to