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

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #819: PARQUET-1915: Add nullify column

gszadovszky commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r515924126



##########
File path: parquet-tools/src/main/java/org/apache/parquet/tools/command/ColumnMaskingCommand.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.tools.command;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.ColumnMasker;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import org.apache.parquet.hadoop.util.ColumnMasker.MaskMode;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+
+public class ColumnMaskingCommand extends ArgsOnlyCommand {

Review comment:
       Is it intentional to have named arguments (e.g. `-c --columns`) in cli but have only positioned ones for tools?

##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompressionConverter.java
##########
@@ -267,5 +275,19 @@ public PageHeader readPageHeader() throws IOException {
     public long getPos() throws IOException {
       return f.getPos();
     }
+
+    public SeekableInputStream getStream() {
+      return f;
+    }
+  }
+
+  public static final class TransParquetFileWriter extends ParquetFileWriter {

Review comment:
       Cannot see the purpose of this class. Its constructor invokes a public (however deprecated) constructor and does not add any functionality.
   If the purpose is to hide the usage of the deprecated constructor (so it will use the default values for a couple of arguments) a simple factory method would be a better choice. Also, I would suggest commenting why you invoke that deprecated constructor.

##########
File path: parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java
##########
@@ -91,5 +92,11 @@
    * @return the number of bytes of memory used to buffer the current data
    */
   long getBufferedSizeInMemory();
+
+  /**
+   * concat all the values writer in the order of repetitionLevel writer, definitionLevel writer, and data writer
+   * @return combined values in the type of ByteInput
+   */
+  BytesInput concatWriters();

Review comment:
       Does not seem to be used anymore.

##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompressionConverter.java
##########
@@ -53,10 +56,15 @@
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT;

Review comment:
       Please, remove the unused imports.




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

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