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