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/12 08:33:32 UTC

[GitHub] [parquet-mr] shangxinli opened a new pull request #819: PARQUET-1915: Add nullify column

shangxinli opened a new pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
     - https://issues.apache.org/jira/browse/PARQUET-XXX
     - In case you are adding a dependency, check if the license complies with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes how to use it.
     - All the public functions and the classes in the PR contain Javadoc that explain what it does
   


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



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

Posted by GitBox <gi...@apache.org>.
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



[GitHub] [parquet-mr] gszadovszky closed pull request #819: PARQUET-1915: Add nullify column

Posted by GitBox <gi...@apache.org>.
gszadovszky closed pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819


   


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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r516773278



##########
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:
       yes, leftovers




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



[GitHub] [parquet-mr] gszadovszky merged pull request #819: PARQUET-1915: Add nullify column

Posted by GitBox <gi...@apache.org>.
gszadovszky merged pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819


   


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



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

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r510113275



##########
File path: parquet-cli/src/main/java/org/apache/parquet/cli/commands/ColumnMaskingCommand.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.cli.BaseCommand;
+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.ColumnMasker.MaskMode;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+@Parameters(commandDescription="Replace columns with masked values and write to a new Parquet file")
+public class ColumnMaskingCommand extends BaseCommand {
+
+  private ColumnMasker masker;
+
+  public ColumnMaskingCommand(Logger console) {
+    super(console);
+    masker = new ColumnMasker();
+  }
+
+  @Parameter(description = "<mask mode: nullify, hash, redact>")
+  String mode;
+
+  @Parameter(description = "<input parquet file path>")
+  String input;
+
+  @Parameter(description = "<output parquet file path>")
+  String output;
+
+  @Parameter(description = "columns to be repalced with masked value")

Review comment:
       ```suggestion
     @Parameter(description = "columns to be replaced with masked value")
   ```

##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnMasker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.util;
+
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageWriteStore;
+import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+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.CompressionConverter.TransParquetFileWriter;
+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.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMasker {
+  /**
+   *
+   * @param reader Reader of source file
+   * @param writer Writer of destination file
+   * @param meta Metadata of source file
+   * @param schema Schema of source file
+   * @param paths Column Paths need to be masked
+   * @param maskMode Mode to mask
+   * @throws IOException
+   */
+  public void processBlocks(TransParquetFileReader reader, TransParquetFileWriter writer, ParquetMetadata meta,
+                            MessageType schema, List<String> paths, MaskMode maskMode) throws IOException {
+    Set<ColumnPath> nullifyColumns = convertToColumnPaths(paths);
+    int blockIndex = 0;
+    PageReadStore store = reader.readNextRowGroup();
+
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+      List<ColumnChunkMetaData> columnsInOrder = meta.getBlocks().get(blockIndex).getColumns();
+      Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+        Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+      ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema,
+        meta.getFileMetaData().getCreatedBy());
+
+      for (int i = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+        processChunk(descriptor, chunk, crStore, reader, writer, schema, nullifyColumns, maskMode);
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockIndex++;
+    }
+  }
+
+  private void processChunk(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                            TransParquetFileReader reader, TransParquetFileWriter writer, MessageType schema,
+                            Set<ColumnPath> paths, MaskMode maskMode) throws IOException {
+    reader.setStreamPosition(chunk.getStartingPos());
+
+    if (paths.contains(chunk.getPath())) {
+      if (maskMode.equals(MaskMode.NULLIFY)) {
+        Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+        if (repetition.equals(Type.Repetition.REQUIRED)) {
+          throw new IOException("Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+        }
+        nullifyColumn(descriptor, chunk, crStore, writer, schema);
+      } else {
+        throw new UnsupportedOperationException("Only nullify is supported for now");
+      }
+    } else {
+      BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+      ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+      OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+      writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+    }
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                             TransParquetFileWriter writer, MessageType schema) throws IOException {
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    writer.startColumn(descriptor, totalChunkValues, CompressionCodecName.UNCOMPRESSED);
+
+    WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ? WriterVersion.PARQUET_2_0 : WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+      .withWriterVersion(writerVersion)
+      .build();
+    ColumnWriter cWriter = props.newColumnWriteStore(schema, new DummyPageWriterStore()).getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " + chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level, instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+    }
+
+    BytesInput data = cWriter.concatWriters();

Review comment:
       So the concept is to write the nulls into a "hacked" `ColumnWrite`r and extract the bytes from it then write it as a page. This way you write the whole column chunk into one page. This breaks the concept of column-indexes that we write new pages for a couple of rows (default is 20k) even if the pages are small in bytes.
   I think, it would be much better to use a proper `PageWriteStore` instead of a dummy one and let the original implementation decide when to create new pages. This way you do not need to extend the existing interface with `contactWriters` (which also breaks the current concept of the interface that is only a sink for the data).

##########
File path: parquet-cli/src/main/java/org/apache/parquet/cli/commands/ColumnMaskingCommand.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.cli.BaseCommand;
+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.ColumnMasker.MaskMode;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+@Parameters(commandDescription="Replace columns with masked values and write to a new Parquet file")
+public class ColumnMaskingCommand extends BaseCommand {
+
+  private ColumnMasker masker;
+
+  public ColumnMaskingCommand(Logger console) {
+    super(console);
+    masker = new ColumnMasker();
+  }
+
+  @Parameter(description = "<mask mode: nullify, hash, redact>")

Review comment:
       I am not sure it is a good idea the different modes and throw exceptions that some of them are not supported/implemented. We are heading to the next major release and I think it is not a good user experience to have such features list as if they work properly but they are not implemented yet.




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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r516800874



##########
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:
       I will remove




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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r510896852



##########
File path: parquet-cli/src/main/java/org/apache/parquet/cli/commands/ColumnMaskingCommand.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.cli.BaseCommand;
+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.ColumnMasker.MaskMode;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+@Parameters(commandDescription="Replace columns with masked values and write to a new Parquet file")
+public class ColumnMaskingCommand extends BaseCommand {
+
+  private ColumnMasker masker;
+
+  public ColumnMaskingCommand(Logger console) {
+    super(console);
+    masker = new ColumnMasker();
+  }
+
+  @Parameter(description = "<mask mode: nullify, hash, redact>")

Review comment:
       > I am not sure it is a good idea the different modes and throw exceptions that some of them are not supported/implemented. We are heading to the next major release and I think it is not a good user experience to have such features list as if they work properly but they are not implemented yet.
   
   Sounds good! I will remove unsupported ones from the list.




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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r516766997



##########
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:
       Sounds good!




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



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

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


   Re-triggering travis builds...


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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r511212212



##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnMasker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.util;
+
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageWriteStore;
+import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+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.CompressionConverter.TransParquetFileWriter;
+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.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMasker {
+  /**
+   *
+   * @param reader Reader of source file
+   * @param writer Writer of destination file
+   * @param meta Metadata of source file
+   * @param schema Schema of source file
+   * @param paths Column Paths need to be masked
+   * @param maskMode Mode to mask
+   * @throws IOException
+   */
+  public void processBlocks(TransParquetFileReader reader, TransParquetFileWriter writer, ParquetMetadata meta,
+                            MessageType schema, List<String> paths, MaskMode maskMode) throws IOException {
+    Set<ColumnPath> nullifyColumns = convertToColumnPaths(paths);
+    int blockIndex = 0;
+    PageReadStore store = reader.readNextRowGroup();
+
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+      List<ColumnChunkMetaData> columnsInOrder = meta.getBlocks().get(blockIndex).getColumns();
+      Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+        Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+      ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema,
+        meta.getFileMetaData().getCreatedBy());
+
+      for (int i = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+        processChunk(descriptor, chunk, crStore, reader, writer, schema, nullifyColumns, maskMode);
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockIndex++;
+    }
+  }
+
+  private void processChunk(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                            TransParquetFileReader reader, TransParquetFileWriter writer, MessageType schema,
+                            Set<ColumnPath> paths, MaskMode maskMode) throws IOException {
+    reader.setStreamPosition(chunk.getStartingPos());
+
+    if (paths.contains(chunk.getPath())) {
+      if (maskMode.equals(MaskMode.NULLIFY)) {
+        Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+        if (repetition.equals(Type.Repetition.REQUIRED)) {
+          throw new IOException("Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+        }
+        nullifyColumn(descriptor, chunk, crStore, writer, schema);
+      } else {
+        throw new UnsupportedOperationException("Only nullify is supported for now");
+      }
+    } else {
+      BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+      ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+      OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+      writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+    }
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                             TransParquetFileWriter writer, MessageType schema) throws IOException {
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    writer.startColumn(descriptor, totalChunkValues, CompressionCodecName.UNCOMPRESSED);
+
+    WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ? WriterVersion.PARQUET_2_0 : WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+      .withWriterVersion(writerVersion)
+      .build();
+    ColumnWriter cWriter = props.newColumnWriteStore(schema, new DummyPageWriterStore()).getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " + chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level, instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+    }
+
+    BytesInput data = cWriter.concatWriters();

Review comment:
       The problem is we need to make the class ColumnChunkPageWriteStore public because we are under the namespace of org.apache.parquet.hadoop.util which cannot use ColumnChunkPageWriteStore for now. 




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



[GitHub] [parquet-mr] shangxinli commented on pull request #819: PARQUET-1915: Add nullify column

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


   @gszadovszky @Fokko  Do you have time to review this change? 


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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r516772547



##########
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:
       It is a left over. I will remove it. 




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

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



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

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r511835929



##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnMasker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.util;
+
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageWriteStore;
+import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+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.CompressionConverter.TransParquetFileWriter;
+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.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMasker {
+  /**
+   *
+   * @param reader Reader of source file
+   * @param writer Writer of destination file
+   * @param meta Metadata of source file
+   * @param schema Schema of source file
+   * @param paths Column Paths need to be masked
+   * @param maskMode Mode to mask
+   * @throws IOException
+   */
+  public void processBlocks(TransParquetFileReader reader, TransParquetFileWriter writer, ParquetMetadata meta,
+                            MessageType schema, List<String> paths, MaskMode maskMode) throws IOException {
+    Set<ColumnPath> nullifyColumns = convertToColumnPaths(paths);
+    int blockIndex = 0;
+    PageReadStore store = reader.readNextRowGroup();
+
+    while (store != null) {
+      writer.startBlock(store.getRowCount());
+      List<ColumnChunkMetaData> columnsInOrder = meta.getBlocks().get(blockIndex).getColumns();
+      Map<ColumnPath, ColumnDescriptor> descriptorsMap = schema.getColumns().stream().collect(
+        Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+      ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema,
+        meta.getFileMetaData().getCreatedBy());
+
+      for (int i = 0; i < columnsInOrder.size(); i += 1) {
+        ColumnChunkMetaData chunk = columnsInOrder.get(i);
+        ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+        processChunk(descriptor, chunk, crStore, reader, writer, schema, nullifyColumns, maskMode);
+      }
+
+      writer.endBlock();
+      store = reader.readNextRowGroup();
+      blockIndex++;
+    }
+  }
+
+  private void processChunk(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                            TransParquetFileReader reader, TransParquetFileWriter writer, MessageType schema,
+                            Set<ColumnPath> paths, MaskMode maskMode) throws IOException {
+    reader.setStreamPosition(chunk.getStartingPos());
+
+    if (paths.contains(chunk.getPath())) {
+      if (maskMode.equals(MaskMode.NULLIFY)) {
+        Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition();
+        if (repetition.equals(Type.Repetition.REQUIRED)) {
+          throw new IOException("Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified");
+        }
+        nullifyColumn(descriptor, chunk, crStore, writer, schema);
+      } else {
+        throw new UnsupportedOperationException("Only nullify is supported for now");
+      }
+    } else {
+      BloomFilter bloomFilter = reader.readBloomFilter(chunk);
+      ColumnIndex columnIndex = reader.readColumnIndex(chunk);
+      OffsetIndex offsetIndex = reader.readOffsetIndex(chunk);
+      writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex);
+    }
+  }
+
+  private void nullifyColumn(ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ColumnReadStoreImpl crStore,
+                             TransParquetFileWriter writer, MessageType schema) throws IOException {
+    long totalChunkValues = chunk.getValueCount();
+    int dMax = descriptor.getMaxDefinitionLevel();
+    ColumnReader cReader = crStore.getColumnReader(descriptor);
+
+    writer.startColumn(descriptor, totalChunkValues, CompressionCodecName.UNCOMPRESSED);
+
+    WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ? WriterVersion.PARQUET_2_0 : WriterVersion.PARQUET_1_0;
+    ParquetProperties props = ParquetProperties.builder()
+      .withWriterVersion(writerVersion)
+      .build();
+    ColumnWriter cWriter = props.newColumnWriteStore(schema, new DummyPageWriterStore()).getColumnWriter(descriptor);
+
+    for (int i = 0; i < totalChunkValues; i++) {
+      int rlvl = cReader.getCurrentRepetitionLevel();
+      int dlvl = cReader.getCurrentDefinitionLevel();
+      if (dlvl == dMax) {
+        // since we checked ether optional or repeated, dlvl should be > 0
+        if (dlvl == 0) {
+          throw new IOException("definition level is detected to be 0 for column " + chunk.getPath().toDotString() + " to be nullified");
+        }
+        // we just write one null for the whole list at the top level, instead of nullify the elements in the list one by one
+        if (rlvl == 0) {
+          cWriter.writeNull(rlvl, dlvl - 1);
+        }
+      } else {
+        cWriter.writeNull(rlvl, dlvl);
+      }
+    }
+
+    BytesInput data = cWriter.concatWriters();

Review comment:
       We already have a lot of classes that are public because of we cannot fine-tune the java visibility but are for internal use only. I would suggest make it public if it is required for your design and comment that it is not for public use. You may want to use the yetus' `@Private` annotation as well.




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



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

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #819:
URL: https://github.com/apache/parquet-mr/pull/819#discussion_r516801981



##########
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:
       Yes. I added the named arguments for cli because of PARQUET-1910. For tools, I see no other commands added. So just follow the pattern. 




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