From f5144b29aaf1d81e5b683494a65f95e0ca6887ea Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Jan 2024 14:22:27 -0800 Subject: [PATCH 01/57] add initial ParquetJoiner implementation --- .../parquet/hadoop/join/JoinOptions.java | 304 +++++++++++++ .../parquet/hadoop/join/ParquetJoiner.java | 427 ++++++++++++++++++ .../parquet/hadoop/join/ParquetJoinTest.java | 248 ++++++++++ 3 files changed, 979 insertions(+) create mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java create mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java create mode 100644 parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java new file mode 100644 index 0000000000..bffe563e37 --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java @@ -0,0 +1,304 @@ +/* + * 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.join; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.Preconditions; +import org.apache.parquet.conf.HadoopParquetConfiguration; +import org.apache.parquet.conf.ParquetConfiguration; +import org.apache.parquet.crypto.FileEncryptionProperties; +import org.apache.parquet.hadoop.IndexCache; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.rewrite.MaskMode; +import org.apache.parquet.hadoop.util.ConfigurationUtil; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.OutputFile; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * A set of options to create a ParquetJoiner. + */ +public class JoinOptions { + + private final ParquetConfiguration conf; + private final List inputFilesL; + private final List inputFilesR; + private final OutputFile outputFile; + private final IndexCache.CacheStrategy indexCacheStrategy; + + private JoinOptions( + ParquetConfiguration conf, + List inputFilesL, + List inputFilesR, + OutputFile outputFile, + IndexCache.CacheStrategy indexCacheStrategy) { + this.conf = conf; + this.inputFilesL = inputFilesL; + this.inputFilesR = inputFilesR; + this.outputFile = outputFile; + this.indexCacheStrategy = indexCacheStrategy; + } + + /** + * Gets the {@link Configuration} part of the join options. + * + * @return the associated {@link Configuration} + */ + public Configuration getConf() { + return ConfigurationUtil.createHadoopConfiguration(conf); + } + + /** + * Gets the {@link ParquetConfiguration} part of the join options. + * + * @return the associated {@link ParquetConfiguration} + */ + public ParquetConfiguration getParquetConfiguration() { + return conf; + } + + private List getInputFiles(List in) { + return in.stream() + .map(f -> { + if (f instanceof HadoopOutputFile) { + HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) f; + return new Path(hadoopOutputFile.getPath()); + } else { + throw new RuntimeException("The input files do not all have an associated Hadoop Path."); + } + }) + .collect(Collectors.toList()); + } + + /** + * Gets the input {@link Path}s for the join if they exist for all input files, + * otherwise throws a {@link RuntimeException}. + * + * @return a {@link List} of the associated input {@link Path}s + */ + public List getInputFilesL() { + return getInputFiles(inputFilesL); + } + + public List getInputFilesR() { + return getInputFiles(inputFilesR); + } + + /** + * Gets the {@link InputFile}s for the join. + * + * @return a {@link List} of the associated {@link InputFile}s + */ + public List getParquetInputFilesL() { + return inputFilesL; + } + + public List getParquetInputFilesR() { + return inputFilesR; + } + + /** + * Get the {@link Path} for the join if it exists, otherwise throws a {@link RuntimeException}. + * + * @return the associated {@link Path} if it exists + */ + public Path getOutputFile() { + if (outputFile instanceof HadoopOutputFile) { + HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) outputFile; + return new Path(hadoopOutputFile.getPath()); + } else { + throw new RuntimeException("The output file does not have an associated Hadoop Path."); + } + } + + /** + * Get the {@link OutputFile} for the join. + * + * @return the associated {@link OutputFile} + */ + public OutputFile getParquetOutputFile() { + return outputFile; + } + + public IndexCache.CacheStrategy getIndexCacheStrategy() { + return indexCacheStrategy; + } + + // Builder to create a JoinerOptions. + public static class Builder { + private final ParquetConfiguration conf; + private final List inputFilesL; + private final List inputFilesR; + private final OutputFile outputFile; + private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; + + /** + * Create a builder to create a RewriterOptions. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFile input file path to read from + * @param outputFile output file path to rewrite to + */ + public Builder(Configuration conf, Path inputFileL, Path inputFileR, Path outputFile) { + this( + new HadoopParquetConfiguration(conf), + HadoopInputFile.fromPathUnchecked(inputFileL, conf), + HadoopInputFile.fromPathUnchecked(inputFileR, conf), + HadoopOutputFile.fromPathUnchecked(outputFile, conf)); + } + + /** + * Create a builder to create a RewriterOptions. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFile input file to read from + * @param outputFile output file to rewrite to + */ + public Builder(ParquetConfiguration conf, InputFile inputFileL, InputFile inputFileR, OutputFile outputFile) { + this(conf, Collections.singletonList(inputFileL), Collections.singletonList(inputFileR), outputFile); + } + + /** + * Create a builder to create a RewriterOptions. + *

+ * Please note that if merging more than one file, the schema of all files must be the same. + * Otherwise, the rewrite will fail. + *

+ * The rewrite will keep original row groups from all input files. This may not be optimal + * if row groups are very small and will not solve small file problems. Instead, it will + * make it worse to have a large file footer in the output file. + * TODO: support rewrite by record to break the original row groups into reasonable ones. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFilesL list of input file paths to read from + * @param outputFile output file path to rewrite to + */ + public Builder(Configuration conf, List inputFilesL, List inputFilesR, Path outputFile) { + this.conf = new HadoopParquetConfiguration(conf); + this.inputFilesL = new ArrayList<>(inputFilesL.size()); + for (Path inputFile : inputFilesL) { + this.inputFilesL.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); + } + this.inputFilesR = new ArrayList<>(inputFilesR.size()); + for (Path inputFile : inputFilesR) { + this.inputFilesR.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); + } + this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); + } + + /** + * Create a builder to create a RewriterOptions. + *

+ * Please note that if merging more than one file, the schema of all files must be the same. + * Otherwise, the rewrite will fail. + *

+ * The rewrite will keep original row groups from all input files. This may not be optimal + * if row groups are very small and will not solve small file problems. Instead, it will + * make it worse to have a large file footer in the output file. + * TODO: support rewrite by record to break the original row groups into reasonable ones. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFilesL list of input file paths to read from + * @param outputFile output file path to rewrite to + */ + public Builder( + ParquetConfiguration conf, + List inputFilesL, + List inputFilesR, + OutputFile outputFile) { + this.conf = conf; + this.inputFilesL = inputFilesL; + this.inputFilesR = inputFilesR; + this.outputFile = outputFile; + } + + /** + * Add an input file to read from. + * + * @param path input file path to read from + * @return self + */ + public Builder addInputFileL(Path path) { + this.inputFilesL.add( + HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); + return this; + } + + public Builder addInputFileR(Path path) { + this.inputFilesR.add( + HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); + return this; + } + + /** + * Add an input file to read from. + * + * @param inputFile input file to read from + * @return self + */ + public Builder addInputFileL(InputFile inputFile) { + this.inputFilesL.add(inputFile); + return this; + } + + public Builder addInputFileR(InputFile inputFile) { + this.inputFilesR.add(inputFile); + return this; + } + + /** + * Set the index(ColumnIndex, Offset and BloomFilter) cache strategy. + *

+ * This could reduce the random seek while rewriting with PREFETCH_BLOCK strategy, NONE by default. + * + * @param cacheStrategy the index cache strategy, supports: {@link IndexCache.CacheStrategy#NONE} or + * {@link IndexCache.CacheStrategy#PREFETCH_BLOCK} + * @return self + */ + public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { + this.indexCacheStrategy = cacheStrategy; + return this; + } + + /** + * Build the RewriterOptions. + * + * @return a RewriterOptions + */ + public JoinOptions build() { + Preconditions.checkArgument(inputFilesL != null && !inputFilesL.isEmpty(), "Input file is required"); + Preconditions.checkArgument(inputFilesR != null && !inputFilesR.isEmpty(), "Input file is required"); + Preconditions.checkArgument(outputFile != null, "Output file is required"); + + return new JoinOptions( + conf, + inputFilesL, + inputFilesR, + outputFile, + indexCacheStrategy); + } + } +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java new file mode 100644 index 0000000000..f698610a3f --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java @@ -0,0 +1,427 @@ +/* + * 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.join; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.*; +import org.apache.parquet.column.impl.ColumnReadStoreImpl; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.conf.ParquetConfiguration; +import org.apache.parquet.hadoop.*; +import org.apache.parquet.hadoop.metadata.*; +import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.api.Binary; +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.Type; +import org.apache.parquet.schema.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.Closeable; +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +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.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE; +import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT; + +public class ParquetJoiner implements Closeable { + + // Key to store original writer version in the file key-value metadata + public static final String ORIGINAL_CREATED_BY_KEY = "original.created.by"; + private static final Logger LOG = LoggerFactory.getLogger(ParquetJoiner.class); + private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2; + private final byte[] pageBuffer = new byte[pageBufferSize]; + + // Configurations for the new file + private final Map extraMetaData; + // Writer to rewrite the input files + private final ParquetFileWriter writer; + + // Reader and relevant states of the in-processing input file + private final Queue inputFilesL; + private final Queue inputFilesR; + // Schema of input files (should be the same) and to write to the output file + private final MessageType schema; + private final MessageType schemaL; + private final MessageType schemaR; + private final Map descriptorsMapL; + private final Map descriptorsMapR; + // created_by information of current reader being processed + private String originalCreatedBy = ""; + // Unique created_by information from all input files +// private final Set allOriginalCreatedBys = new HashSet<>(); + // The index cache strategy + private final IndexCache.CacheStrategy indexCacheStrategy; + + public ParquetJoiner(JoinOptions options) throws IOException { + ParquetConfiguration conf = options.getParquetConfiguration(); + OutputFile outFile = options.getParquetOutputFile(); + this.inputFilesL = getFileReader(options.getParquetInputFilesL(), conf); + this.inputFilesR = getFileReader(options.getParquetInputFilesR(), conf); + + Map map = new HashMap<>(); + map.put( + ORIGINAL_CREATED_BY_KEY, + String.join( + "\n", + Stream.concat(inputFilesL.stream(), inputFilesR.stream()) + .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .collect(Collectors.toSet()) + ) + ); + this.extraMetaData = Collections.unmodifiableMap(map); + + LOG.info("Start rewriting {} input file(s) {} to {}", inputFilesL.size(), options.getParquetInputFilesL(), outFile); // TODO + + this.schemaL = getInputFilesSchema(inputFilesL); + this.schemaR = getInputFilesSchema(inputFilesR); + + Set fieldNamesR = schemaR.getFields().stream().map(Type::getName).collect(Collectors.toSet()); + List fields = Stream.concat( + schemaL.getFields().stream().filter(x -> !fieldNamesR.contains(x.getName())), + schemaR.getFields().stream() + ).collect(Collectors.toList()); + this.schema = new MessageType(schemaL.getName(), fields); + + this.descriptorsMapL = schemaL.getColumns().stream() + .filter(x -> x.getPath().length == 0 || !fieldNamesR.contains(x.getPath()[0])) + .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + this.descriptorsMapR = schemaR.getColumns().stream() + .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + this.indexCacheStrategy = options.getIndexCacheStrategy(); + + long rowCountL = inputFilesL.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + long rowCountR = inputFilesR.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + if (rowCountL != rowCountR) { + throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); + } + + ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; + writer = new ParquetFileWriter( + outFile, + schema, + writerMode, + DEFAULT_BLOCK_SIZE, + MAX_PADDING_SIZE_DEFAULT, + DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, + DEFAULT_STATISTICS_TRUNCATE_LENGTH, + ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); + writer.start(); + } + + // Open all input files to validate their schemas are compatible to merge + private Queue getFileReader(List inputFiles, ParquetConfiguration conf) { + Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); + LinkedList inputFileReaders = new LinkedList<>(); + for (InputFile inputFile : inputFiles) { + try { + TransParquetFileReader reader = new TransParquetFileReader( + inputFile, ParquetReadOptions.builder(conf).build()); + inputFileReaders.add(reader); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to open input file: " + inputFile, e); + } + } + return inputFileReaders; + } + + private MessageType getInputFilesSchema(Queue inputFileReaders) { + MessageType schema = null; + for (TransParquetFileReader reader : inputFileReaders) { + MessageType newSchema = reader.getFooter().getFileMetaData().getSchema(); + if (schema == null) { + schema = newSchema; + } else { + // Now we enforce equality of schemas from input files for simplicity. + if (!schema.equals(newSchema)) { + String file = reader.getFile(); + LOG.error( + "Input files have different schemas, expect: {}, input: {}, current file: {}", + schema, + newSchema, + file); + throw new InvalidSchemaException( + "Input files have different schemas, current file: " + file); + } + } + } + return schema; + } + + @Override + public void close() throws IOException { + writer.end(extraMetaData); + } + +// private static class ColumnReaderIterator implements Iterator { +// +// @Override +// public boolean hasNext() { +// return false; +// } +// +// @Override +// public ColumnReader next() { +// return null; +// } +// } + + public void processBlocks() throws IOException { + int numBlocksRewritten = 0; +// new ColumnReaderIterator(); + Map colReadersR = new HashMap<>(); + int blockIdR = 0; + int writtenInBlock = 0; + while (!inputFilesL.isEmpty()) { + + + TransParquetFileReader readerL = inputFilesL.poll(); + IndexCache indexCacheL = IndexCache.create(readerL, descriptorsMapL.keySet(), indexCacheStrategy, true); + LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", readerL.getFile(), inputFilesL.size()); + ParquetMetadata metaL = readerL.getFooter(); + for (int blockId = 0; blockId < metaL.getBlocks().size(); blockId++) { + BlockMetaData blockMetaDataL = metaL.getBlocks().get(blockId); + writer.startBlock(blockMetaDataL.getRowCount()); + + + indexCacheL.setBlockMetadata(blockMetaDataL); + List chunksL = blockMetaDataL.getColumns(); + for (ColumnChunkMetaData chunk : chunksL) { + // TODO add that detail to docs + if (chunk.isEncrypted()) { // If a column is encrypted we simply throw exception. + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); + } + ColumnDescriptor descriptorL = descriptorsMapL.get(chunk.getPath()); + if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join + readerL.setStreamPosition(chunk.getStartingPos()); + BloomFilter bloomFilter = indexCacheL.getBloomFilter(chunk); + ColumnIndex columnIndex = indexCacheL.getColumnIndex(chunk); + OffsetIndex offsetIndex = indexCacheL.getOffsetIndex(chunk); + writer.appendColumnChunk(descriptorL, readerL.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); + } + } + + // TODO < ------------- Left and Right descriptorL must be alligned? + + if (inputFilesR.isEmpty()) { + throw new RuntimeException(""); // TODO + } + ParquetMetadata metaR = inputFilesR.peek().getFooter(); + long writeLeft = blockMetaDataL.getRowCount(); + while (writeLeft > 0) { + // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + BlockMetaData blockMetaDataR = metaR.getBlocks().get(blockIdR); + List chunksR = blockMetaDataR.getColumns(); + long leftInBlock = blockMetaDataR.getRowCount() - writtenInBlock; + long writeFromBlock = Math.min(writeLeft, leftInBlock); + for (ColumnChunkMetaData chunkR : chunksR) { + // If a column is encrypted we simply throw exception. // TODO add that detail to docs + if (chunkR.isEncrypted()) { + throw new IOException("Column " + chunkR.getPath().toDotString() + " is encrypted"); + } + // This column has been pruned. + ColumnDescriptor descriptorR = descriptorsMapR.get(chunkR.getPath()); + TransParquetFileReader readerR = inputFilesR.peek(); + if (!colReadersR.containsKey(descriptorR)) { + PageReadStore pageReadStore = readerR.readRowGroup(blockIdR); + ColumnReadStoreImpl crStore = + new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, originalCreatedBy); + ColumnReader cReader = crStore.getColumnReader(descriptorR); + colReadersR.put(descriptorR, cReader); + } + ColumnReader colReaderR = colReadersR.get(descriptorR); + buildChunks(descriptorR, chunkR, colReaderR, writer, schemaR, writeFromBlock, numBlocksRewritten); + } + writeLeft -= Math.min(writeLeft, blockMetaDataR.getRowCount()); // TODO add exception for empty right schema so we don't fall with exception here? + writtenInBlock += writeFromBlock; + if (writeLeft > 0) { + blockIdR++; + } + if (blockIdR == metaR.getBlocks().size()) { + inputFilesR.poll(); + blockIdR = 0; + writtenInBlock = 0; + } + } + + + writer.endBlock(); + numBlocksRewritten++; + } + } + } + + public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException { + byte[] data; + if (length > pageBufferSize) { + data = new byte[length]; + } else { + data = pageBuffer; + } + reader.blockRead(data, 0, length); + return BytesInput.from(data, 0, length); + } + + private void buildChunks( + ColumnDescriptor descriptor, + ColumnChunkMetaData chunk, + ColumnReader cReader, + ParquetFileWriter writer, + MessageType schema, + long rowsToWrite, + int numBlocksRewritten) + throws IOException { + + ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = + ParquetProperties.builder().withWriterVersion(writerVersion).build(); + CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); + CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + + // Create new schema that only has the current column + MessageType newSchema = newSchema(schema, descriptor); + ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( + compressor, + newSchema, + props.getAllocator(), + props.getColumnIndexTruncateLength(), + props.getPageWriteChecksumEnabled(), + writer.getEncryptor(), + numBlocksRewritten); + ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore); + ColumnWriter cWriter = cStore.getColumnWriter(descriptor); + Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; + + int rowCount = 0; + while (rowCount < rowsToWrite) { +// for (int i = 0; i < rowsToWrite; i++) { + int rlvl = cReader.getCurrentRepetitionLevel(); + int dlvl = cReader.getCurrentDefinitionLevel(); + if (rlvl == 0) { + if (rowCount > 0) { + cStore.endRecord(); + } + rowCount++; + } + if (columnType == Integer.TYPE) { + cWriter.write(cReader.getInteger(), rlvl, dlvl); + } else if (columnType == Long.TYPE) { + cWriter.write(cReader.getLong(), rlvl, dlvl); + } else if (columnType == Float.TYPE) { + cWriter.write(cReader.getFloat(), rlvl, dlvl); + } else if (columnType == Double.TYPE) { + cWriter.write(cReader.getDouble(), rlvl, dlvl); + } else if (columnType == Binary.class) { + cWriter.write(cReader.getBinary(), rlvl, dlvl); + } else if (columnType == Boolean.TYPE) { + cWriter.write(cReader.getBoolean(), rlvl, dlvl); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported column java class: %s", columnType.toString())); + } + cReader.consume(); + } + cStore.endRecord(); + + cStore.flush(); + cPageStore.flushToFileWriter(writer); + + cStore.close(); + cWriter.close(); + } + + private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) { + String[] path = descriptor.getPath(); + Type type = schema.getType(path); + if (path.length == 1) { + return new MessageType(schema.getName(), type); + } + + for (Type field : schema.getFields()) { + if (!field.isPrimitive()) { + Type newType = extractField(field.asGroupType(), type); + if (newType != null) { + return new MessageType(schema.getName(), newType); + } + } + } + + // We should never hit this because 'type' is returned by schema.getType(). + throw new RuntimeException("No field is found"); + } + + private Type extractField(GroupType candidate, Type targetField) { + if (targetField.equals(candidate)) { + return targetField; + } + + // In case 'type' is a descendants of candidate + for (Type field : candidate.asGroupType().getFields()) { + if (field.isPrimitive()) { + if (field.equals(targetField)) { + return new GroupType(candidate.getRepetition(), candidate.getName(), targetField); + } + } else { + Type tempField = extractField(field.asGroupType(), targetField); + if (tempField != null) { + return tempField; + } + } + } + + return null; + } + + private static final class DummyGroupConverter extends GroupConverter { + @Override + public void start() {} + + @Override + public void end() {} + + @Override + public Converter getConverter(int fieldIndex) { + return new DummyConverter(); + } + } + + private static final class DummyConverter extends PrimitiveConverter { + @Override + public GroupConverter asGroupConverter() { + return new DummyGroupConverter(); + } + } + +} diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java new file mode 100644 index 0000000000..e6c7226bd4 --- /dev/null +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java @@ -0,0 +1,248 @@ +/* + * 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.join; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Version; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.conf.ParquetConfiguration; +import org.apache.parquet.conf.PlainParquetConfiguration; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.FileEncryptionProperties; +import org.apache.parquet.crypto.ParquetCipher; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +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.IndexCache; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.metadata.*; +import org.apache.parquet.hadoop.rewrite.MaskMode; +import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; +import org.apache.parquet.hadoop.util.*; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.InvalidRecordException; +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.SeekableInputStream; +import org.apache.parquet.schema.*; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; + +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; +import static org.apache.parquet.schema.Type.Repetition.*; +import static org.junit.Assert.*; + +@RunWith(Parameterized.class) +public class ParquetJoinTest { + + private final int numRecord = 100000; + private final Configuration conf = new Configuration(); + private final ParquetConfiguration parquetConf = new PlainParquetConfiguration(); + private final ParquetProperties.WriterVersion writerVersion; + private final IndexCache.CacheStrategy indexCacheStrategy; + private final boolean usingHadoop; + + private List inputFilesL = null; + private List inputFilesR = null; + private String outputFile = null; + private ParquetJoiner joiner = null; + + @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") + public static Object[][] parameters() { + return new Object[][] { +// {"v1", "NONE", true}, +// {"v1", "PREFETCH_BLOCK", true}, +// {"v2", "NONE", true}, +// {"v2", "PREFETCH_BLOCK", true}, +// {"v1", "NONE", false}, +// {"v1", "PREFETCH_BLOCK", false}, +// {"v2", "NONE", false}, + {"v2", "PREFETCH_BLOCK", false} + }; + } + + @Before + public void setUp() { + outputFile = TestFileBuilder.createTempFile("test"); + } + + public ParquetJoinTest(String writerVersion, String indexCacheStrategy, boolean usingHadoop) { + this.writerVersion = ParquetProperties.WriterVersion.fromString(writerVersion); + this.indexCacheStrategy = IndexCache.CacheStrategy.valueOf(indexCacheStrategy); + this.usingHadoop = usingHadoop; + } + + @Test + public void testMergeTwoFilesOnly() throws Exception { + testMultipleInputFilesSetup(); + + // Only merge two files but do not change anything. + List inputPathsL = new ArrayList<>(); + for (EncryptionTestFile inputFile : inputFilesL) { + inputPathsL.add(new Path(inputFile.getFileName())); + } + List inputPathsR = new ArrayList<>(); + for (EncryptionTestFile inputFile : inputFilesR) { + inputPathsR.add(new Path(inputFile.getFileName())); + } + JoinOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); + JoinOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); + + joiner = new ParquetJoiner(options); + joiner.processBlocks(); + joiner.close(); + + // Verify the schema are not changed + ParquetMetadata pmd = + ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); + MessageType schema = pmd.getFileMetaData().getSchema(); + MessageType expectSchema = createSchemaL(); +// assertEquals(expectSchema, schema); + + // Verify the merged data are not changed + validateColumnData(null); + } + + private void testMultipleInputFilesSetup() throws IOException { + inputFilesL = Lists.newArrayList(); + inputFilesL.add(new TestFileBuilder(conf, createSchemaL()) + .withNumRecord(numRecord) + .withCodec("GZIP") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); + inputFilesR = Lists.newArrayList(); + inputFilesR.add(new TestFileBuilder(conf, createSchemaR()) + .withNumRecord(numRecord) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); + } + + private MessageType createSchemaL() { + return new MessageType( + "schema", + new PrimitiveType(OPTIONAL, INT64, "DocId"), + new PrimitiveType(REQUIRED, BINARY, "Name"), + new PrimitiveType(OPTIONAL, BINARY, "Gender"), + new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); + } + + + private MessageType createSchemaR() { + return new MessageType( + "schema", + new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction"), + new GroupType( + OPTIONAL, + "Links", + new PrimitiveType(REPEATED, BINARY, "Backward"), + new PrimitiveType(REPEATED, BINARY, "Forward"))); + } + + private void validateColumnData( + FileDecryptionProperties fileDecryptionProperties) + throws IOException { + ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), new Path(outputFile)) + .withConf(conf) + .withDecryption(fileDecryptionProperties) + .build(); + + // Get total number of rows from input files + int totalRows = 0; + for (EncryptionTestFile inputFile : inputFilesL) { + totalRows += inputFile.getFileContent().length; + } + + for (int i = 0; i < totalRows; i++) { + Group group = reader.read(); + assertNotNull(group); + + SimpleGroup expectGroupL = inputFilesL.get(i / numRecord).getFileContent()[i % numRecord]; + SimpleGroup expectGroupR = inputFilesR.get(i / numRecord).getFileContent()[i % numRecord]; + + assertEquals(group.getLong("DocId", 0), expectGroupL.getLong("DocId", 0)); + assertArrayEquals( + group.getBinary("Name", 0).getBytes(), + expectGroupL.getBinary("Name", 0).getBytes()); + assertArrayEquals( + group.getBinary("Gender", 0).getBytes(), + expectGroupL.getBinary("Gender", 0).getBytes()); + assertEquals(group.getFloat("FloatFraction", 0), expectGroupL.getFloat("FloatFraction", 0), 0); + assertEquals(group.getDouble("DoubleFraction", 0), expectGroupR.getDouble("DoubleFraction", 0), 0); + Group subGroup = group.getGroup("Links", 0); + assertArrayEquals( + subGroup.getBinary("Backward", 0).getBytes(), + expectGroupR + .getGroup("Links", 0) + .getBinary("Backward", 0) + .getBytes()); + assertArrayEquals( + subGroup.getBinary("Forward", 0).getBytes(), + expectGroupR + .getGroup("Links", 0) + .getBinary("Forward", 0) + .getBytes()); + } + + reader.close(); + } + + + + + private JoinOptions.Builder createBuilder(List inputPathsL, List inputPathsR) throws IOException { + JoinOptions.Builder builder; + if (usingHadoop) { + Path outputPath = new Path(outputFile); + builder = new JoinOptions.Builder(conf, inputPathsL, inputPathsR, outputPath); + } else { + OutputFile outputPath = HadoopOutputFile.fromPath(new Path(outputFile), conf); + List inputsL = inputPathsL.stream() + .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) + .collect(Collectors.toList()); + List inputsR = inputPathsR.stream() + .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) + .collect(Collectors.toList()); + builder = new JoinOptions.Builder(parquetConf, inputsL, inputsR, outputPath); + } + return builder; + } + +} From 01a08ddf0c606186343098b9051dfd23bfd91d07 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 31 Jan 2024 17:18:51 -0800 Subject: [PATCH 02/57] add initial ParquetJoiner implementation --- .../java/org/apache/parquet/hadoop/join/ParquetJoiner.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java index f698610a3f..eb8fa531a5 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java @@ -302,6 +302,7 @@ private void buildChunks( int numBlocksRewritten) throws IOException { + int dMax = descriptor.getMaxDefinitionLevel(); ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() ? ParquetProperties.WriterVersion.PARQUET_2_0 : ParquetProperties.WriterVersion.PARQUET_1_0; @@ -335,7 +336,9 @@ private void buildChunks( } rowCount++; } - if (columnType == Integer.TYPE) { + if (dlvl < dMax) { + cWriter.writeNull(rlvl, dlvl); + } else if (columnType == Integer.TYPE) { cWriter.write(cReader.getInteger(), rlvl, dlvl); } else if (columnType == Long.TYPE) { cWriter.write(cReader.getLong(), rlvl, dlvl); @@ -396,7 +399,7 @@ private Type extractField(GroupType candidate, Type targetField) { } else { Type tempField = extractField(field.asGroupType(), targetField); if (tempField != null) { - return tempField; + return new GroupType(candidate.getRepetition(), candidate.getName(), tempField); } } } From 7ae35059ae9801e0ffb7f9a0dc825621dbc37ecc Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Fri, 16 Feb 2024 18:02:56 -0800 Subject: [PATCH 03/57] refactor ParquetJoiner implementation --- .../parquet/hadoop/join/JoinOptions.java | 188 +------ .../parquet/hadoop/join/ParquetJoiner.java | 476 +++++++++--------- .../parquet/hadoop/join/ParquetJoinTest.java | 56 +-- 3 files changed, 274 insertions(+), 446 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java index bffe563e37..18433fcd4b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java @@ -23,36 +23,27 @@ import org.apache.parquet.Preconditions; import org.apache.parquet.conf.HadoopParquetConfiguration; import org.apache.parquet.conf.ParquetConfiguration; -import org.apache.parquet.crypto.FileEncryptionProperties; import org.apache.parquet.hadoop.IndexCache; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; -import org.apache.parquet.hadoop.rewrite.MaskMode; import org.apache.parquet.hadoop.util.ConfigurationUtil; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.hadoop.util.HadoopOutputFile; import org.apache.parquet.io.InputFile; import org.apache.parquet.io.OutputFile; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; -/** - * A set of options to create a ParquetJoiner. - */ public class JoinOptions { private final ParquetConfiguration conf; private final List inputFilesL; - private final List inputFilesR; + private final List> inputFilesR; private final OutputFile outputFile; private final IndexCache.CacheStrategy indexCacheStrategy; private JoinOptions( ParquetConfiguration conf, List inputFilesL, - List inputFilesR, + List> inputFilesR, OutputFile outputFile, IndexCache.CacheStrategy indexCacheStrategy) { this.conf = conf; @@ -62,20 +53,10 @@ private JoinOptions( this.indexCacheStrategy = indexCacheStrategy; } - /** - * Gets the {@link Configuration} part of the join options. - * - * @return the associated {@link Configuration} - */ public Configuration getConf() { return ConfigurationUtil.createHadoopConfiguration(conf); } - /** - * Gets the {@link ParquetConfiguration} part of the join options. - * - * @return the associated {@link ParquetConfiguration} - */ public ParquetConfiguration getParquetConfiguration() { return conf; } @@ -93,38 +74,14 @@ private List getInputFiles(List in) { .collect(Collectors.toList()); } - /** - * Gets the input {@link Path}s for the join if they exist for all input files, - * otherwise throws a {@link RuntimeException}. - * - * @return a {@link List} of the associated input {@link Path}s - */ - public List getInputFilesL() { - return getInputFiles(inputFilesL); - } - - public List getInputFilesR() { - return getInputFiles(inputFilesR); - } - - /** - * Gets the {@link InputFile}s for the join. - * - * @return a {@link List} of the associated {@link InputFile}s - */ public List getParquetInputFilesL() { return inputFilesL; } - public List getParquetInputFilesR() { + public List> getParquetInputFilesR() { return inputFilesR; } - /** - * Get the {@link Path} for the join if it exists, otherwise throws a {@link RuntimeException}. - * - * @return the associated {@link Path} if it exists - */ public Path getOutputFile() { if (outputFile instanceof HadoopOutputFile) { HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) outputFile; @@ -134,11 +91,6 @@ public Path getOutputFile() { } } - /** - * Get the {@link OutputFile} for the join. - * - * @return the associated {@link OutputFile} - */ public OutputFile getParquetOutputFile() { return outputFile; } @@ -147,87 +99,30 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { return indexCacheStrategy; } - // Builder to create a JoinerOptions. public static class Builder { private final ParquetConfiguration conf; private final List inputFilesL; - private final List inputFilesR; + private final List> inputFilesR; private final OutputFile outputFile; private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; - /** - * Create a builder to create a RewriterOptions. - * - * @param conf configuration for reading from input files and writing to output file - * @param inputFile input file path to read from - * @param outputFile output file path to rewrite to - */ - public Builder(Configuration conf, Path inputFileL, Path inputFileR, Path outputFile) { - this( - new HadoopParquetConfiguration(conf), - HadoopInputFile.fromPathUnchecked(inputFileL, conf), - HadoopInputFile.fromPathUnchecked(inputFileR, conf), - HadoopOutputFile.fromPathUnchecked(outputFile, conf)); - } - - /** - * Create a builder to create a RewriterOptions. - * - * @param conf configuration for reading from input files and writing to output file - * @param inputFile input file to read from - * @param outputFile output file to rewrite to - */ - public Builder(ParquetConfiguration conf, InputFile inputFileL, InputFile inputFileR, OutputFile outputFile) { - this(conf, Collections.singletonList(inputFileL), Collections.singletonList(inputFileR), outputFile); - } - - /** - * Create a builder to create a RewriterOptions. - *

- * Please note that if merging more than one file, the schema of all files must be the same. - * Otherwise, the rewrite will fail. - *

- * The rewrite will keep original row groups from all input files. This may not be optimal - * if row groups are very small and will not solve small file problems. Instead, it will - * make it worse to have a large file footer in the output file. - * TODO: support rewrite by record to break the original row groups into reasonable ones. - * - * @param conf configuration for reading from input files and writing to output file - * @param inputFilesL list of input file paths to read from - * @param outputFile output file path to rewrite to - */ - public Builder(Configuration conf, List inputFilesL, List inputFilesR, Path outputFile) { + public Builder(Configuration conf, List inputFilesL, List> inputFilesR, Path outputFile) { this.conf = new HadoopParquetConfiguration(conf); - this.inputFilesL = new ArrayList<>(inputFilesL.size()); - for (Path inputFile : inputFilesL) { - this.inputFilesL.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); - } - this.inputFilesR = new ArrayList<>(inputFilesR.size()); - for (Path inputFile : inputFilesR) { - this.inputFilesR.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); - } + this.inputFilesL = inputFilesL + .stream() + .map(x -> HadoopInputFile.fromPathUnchecked(x, conf)) + .collect(Collectors.toList()); + this.inputFilesR = inputFilesR + .stream() + .map(x -> x.stream().map(y -> (InputFile) HadoopInputFile.fromPathUnchecked(y, conf)).collect(Collectors.toList())) + .collect(Collectors.toList()); this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } - /** - * Create a builder to create a RewriterOptions. - *

- * Please note that if merging more than one file, the schema of all files must be the same. - * Otherwise, the rewrite will fail. - *

- * The rewrite will keep original row groups from all input files. This may not be optimal - * if row groups are very small and will not solve small file problems. Instead, it will - * make it worse to have a large file footer in the output file. - * TODO: support rewrite by record to break the original row groups into reasonable ones. - * - * @param conf configuration for reading from input files and writing to output file - * @param inputFilesL list of input file paths to read from - * @param outputFile output file path to rewrite to - */ public Builder( ParquetConfiguration conf, List inputFilesL, - List inputFilesR, + List> inputFilesR, OutputFile outputFile) { this.conf = conf; this.inputFilesL = inputFilesL; @@ -235,62 +130,16 @@ public Builder( this.outputFile = outputFile; } - /** - * Add an input file to read from. - * - * @param path input file path to read from - * @return self - */ - public Builder addInputFileL(Path path) { - this.inputFilesL.add( - HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); - return this; - } - - public Builder addInputFileR(Path path) { - this.inputFilesR.add( - HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); - return this; - } - - /** - * Add an input file to read from. - * - * @param inputFile input file to read from - * @return self - */ - public Builder addInputFileL(InputFile inputFile) { - this.inputFilesL.add(inputFile); - return this; - } - - public Builder addInputFileR(InputFile inputFile) { - this.inputFilesR.add(inputFile); - return this; - } - - /** - * Set the index(ColumnIndex, Offset and BloomFilter) cache strategy. - *

- * This could reduce the random seek while rewriting with PREFETCH_BLOCK strategy, NONE by default. - * - * @param cacheStrategy the index cache strategy, supports: {@link IndexCache.CacheStrategy#NONE} or - * {@link IndexCache.CacheStrategy#PREFETCH_BLOCK} - * @return self - */ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { this.indexCacheStrategy = cacheStrategy; return this; } - /** - * Build the RewriterOptions. - * - * @return a RewriterOptions - */ public JoinOptions build() { - Preconditions.checkArgument(inputFilesL != null && !inputFilesL.isEmpty(), "Input file is required"); - Preconditions.checkArgument(inputFilesR != null && !inputFilesR.isEmpty(), "Input file is required"); + // TODO move these check to the ParquetJoiner itself? + Preconditions.checkArgument(inputFilesL != null && !inputFilesL.isEmpty(), "Input fileL can't be NULL or empty"); + Preconditions.checkArgument(inputFilesR != null && !inputFilesR.isEmpty(), "Input fileR can't be NULL or empty"); + Preconditions.checkArgument(inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), "Input fileR elements can't be NULL or empty"); Preconditions.checkArgument(outputFile != null, "Output file is required"); return new JoinOptions( @@ -301,4 +150,5 @@ public JoinOptions build() { indexCacheStrategy); } } + } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java index eb8fa531a5..cd492e5e2e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java @@ -18,10 +18,10 @@ */ package org.apache.parquet.hadoop.join; +import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Preconditions; -import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.*; import org.apache.parquet.column.impl.ColumnReadStoreImpl; import org.apache.parquet.column.page.PageReadStore; @@ -59,8 +59,6 @@ public class ParquetJoiner implements Closeable { // Key to store original writer version in the file key-value metadata public static final String ORIGINAL_CREATED_BY_KEY = "original.created.by"; private static final Logger LOG = LoggerFactory.getLogger(ParquetJoiner.class); - private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2; - private final byte[] pageBuffer = new byte[pageBufferSize]; // Configurations for the new file private final Map extraMetaData; @@ -69,65 +67,61 @@ public class ParquetJoiner implements Closeable { // Reader and relevant states of the in-processing input file private final Queue inputFilesL; - private final Queue inputFilesR; - // Schema of input files (should be the same) and to write to the output file - private final MessageType schema; - private final MessageType schemaL; - private final MessageType schemaR; + private final List columnWritersR = new ArrayList<>(); private final Map descriptorsMapL; - private final Map descriptorsMapR; - // created_by information of current reader being processed - private String originalCreatedBy = ""; - // Unique created_by information from all input files -// private final Set allOriginalCreatedBys = new HashSet<>(); // The index cache strategy private final IndexCache.CacheStrategy indexCacheStrategy; public ParquetJoiner(JoinOptions options) throws IOException { ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile outFile = options.getParquetOutputFile(); - this.inputFilesL = getFileReader(options.getParquetInputFilesL(), conf); - this.inputFilesR = getFileReader(options.getParquetInputFilesR(), conf); - - Map map = new HashMap<>(); - map.put( - ORIGINAL_CREATED_BY_KEY, - String.join( - "\n", - Stream.concat(inputFilesL.stream(), inputFilesR.stream()) - .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) - .collect(Collectors.toSet()) - ) + this.inputFilesL = getFileReaders(options.getParquetInputFilesL(), conf); + List> inputFilesR = options.getParquetInputFilesR() + .stream() + .map(x -> getFileReaders(x, conf)) + .collect(Collectors.toList()); + ensureSameSchema(inputFilesL); + inputFilesR.forEach(this::ensureSameSchema); + LOG.info("Start rewriting {} input file(s) {} to {}", inputFilesL.size(), options.getParquetInputFilesL(), outFile); // TODO add logging for all the files + + this.extraMetaData = ImmutableMap.of( + ORIGINAL_CREATED_BY_KEY, + Stream.concat(inputFilesL.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .reduce((a, b) -> a + "\n" + b) + .orElse("") ); - this.extraMetaData = Collections.unmodifiableMap(map); - LOG.info("Start rewriting {} input file(s) {} to {}", inputFilesL.size(), options.getParquetInputFilesL(), outFile); // TODO + // TODO check that schema on the left and on the right is not identical + MessageType schemaL = inputFilesL.peek().getFooter().getFileMetaData().getSchema(); + List schemaR = inputFilesR + .stream() + .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) + .collect(Collectors.toList()); - this.schemaL = getInputFilesSchema(inputFilesL); - this.schemaR = getInputFilesSchema(inputFilesR); - - Set fieldNamesR = schemaR.getFields().stream().map(Type::getName).collect(Collectors.toSet()); + // TODO check that there is no overlap of fields on the right + Map fieldNamesL = schemaL.getFields().stream().collect(Collectors.toMap(Type::getName, x -> x)); + Map fieldNamesR = schemaR.stream().flatMap(x -> x.getFields().stream()).collect(Collectors.toMap(Type::getName, x -> x)); List fields = Stream.concat( - schemaL.getFields().stream().filter(x -> !fieldNamesR.contains(x.getName())), - schemaR.getFields().stream() + fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can + fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left ).collect(Collectors.toList()); - this.schema = new MessageType(schemaL.getName(), fields); + // Schema of input files (should be the same) and to write to the output file + MessageType schema = new MessageType(schemaL.getName(), fields); this.descriptorsMapL = schemaL.getColumns().stream() - .filter(x -> x.getPath().length == 0 || !fieldNamesR.contains(x.getPath()[0])) - .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - this.descriptorsMapR = schemaR.getColumns().stream() + .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); this.indexCacheStrategy = options.getIndexCacheStrategy(); long rowCountL = inputFilesL.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - long rowCountR = inputFilesR.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + long rowCountR = inputFilesR.stream().flatMap(Collection::stream).mapToLong(ParquetFileReader::getRecordCount).sum(); if (rowCountL != rowCountR) { throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); } ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; - writer = new ParquetFileWriter( + this.writer = new ParquetFileWriter( outFile, schema, writerMode, @@ -136,11 +130,15 @@ public ParquetJoiner(JoinOptions options) throws IOException { DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); - writer.start(); + this.writer.start(); + + for (Queue inFiles: inputFilesR) { + this.columnWritersR.add(new RightColumnWriter(inFiles, writer)); + } } // Open all input files to validate their schemas are compatible to merge - private Queue getFileReader(List inputFiles, ParquetConfiguration conf) { + private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); LinkedList inputFileReaders = new LinkedList<>(); for (InputFile inputFile : inputFiles) { @@ -155,7 +153,7 @@ private Queue getFileReader(List inputFiles, return inputFileReaders; } - private MessageType getInputFilesSchema(Queue inputFileReaders) { + private void ensureSameSchema(Queue inputFileReaders) { MessageType schema = null; for (TransParquetFileReader reader : inputFileReaders) { MessageType newSchema = reader.getFooter().getFileMetaData().getSchema(); @@ -175,7 +173,6 @@ private MessageType getInputFilesSchema(Queue inputFileR } } } - return schema; } @Override @@ -183,248 +180,241 @@ public void close() throws IOException { writer.end(extraMetaData); } -// private static class ColumnReaderIterator implements Iterator { -// -// @Override -// public boolean hasNext() { -// return false; -// } -// -// @Override -// public ColumnReader next() { -// return null; -// } -// } - + // TODO add the test for empty files joins, it should merge schemas public void processBlocks() throws IOException { - int numBlocksRewritten = 0; -// new ColumnReaderIterator(); - Map colReadersR = new HashMap<>(); - int blockIdR = 0; - int writtenInBlock = 0; + int rowGroupIdx = 0; while (!inputFilesL.isEmpty()) { - - - TransParquetFileReader readerL = inputFilesL.poll(); - IndexCache indexCacheL = IndexCache.create(readerL, descriptorsMapL.keySet(), indexCacheStrategy, true); - LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", readerL.getFile(), inputFilesL.size()); - ParquetMetadata metaL = readerL.getFooter(); - for (int blockId = 0; blockId < metaL.getBlocks().size(); blockId++) { - BlockMetaData blockMetaDataL = metaL.getBlocks().get(blockId); - writer.startBlock(blockMetaDataL.getRowCount()); - - - indexCacheL.setBlockMetadata(blockMetaDataL); - List chunksL = blockMetaDataL.getColumns(); + TransParquetFileReader reader = inputFilesL.poll(); + IndexCache indexCache = IndexCache.create(reader, descriptorsMapL.keySet(), indexCacheStrategy, true); + LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFilesL.size()); + List blocks = reader.getFooter().getBlocks(); + for (BlockMetaData blockMetaData: blocks) { + writer.startBlock(blockMetaData.getRowCount()); + + // Writing the left side + indexCache.setBlockMetadata(blockMetaData); + List chunksL = blockMetaData.getColumns(); for (ColumnChunkMetaData chunk : chunksL) { - // TODO add that detail to docs - if (chunk.isEncrypted()) { // If a column is encrypted we simply throw exception. + if (chunk.isEncrypted()) { // TODO add that detail to docs throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); } ColumnDescriptor descriptorL = descriptorsMapL.get(chunk.getPath()); if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join - readerL.setStreamPosition(chunk.getStartingPos()); - BloomFilter bloomFilter = indexCacheL.getBloomFilter(chunk); - ColumnIndex columnIndex = indexCacheL.getColumnIndex(chunk); - OffsetIndex offsetIndex = indexCacheL.getOffsetIndex(chunk); - writer.appendColumnChunk(descriptorL, readerL.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); + reader.setStreamPosition(chunk.getStartingPos()); + BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); + ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); + OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); + writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } } - // TODO < ------------- Left and Right descriptorL must be alligned? - - if (inputFilesR.isEmpty()) { - throw new RuntimeException(""); // TODO - } - ParquetMetadata metaR = inputFilesR.peek().getFooter(); - long writeLeft = blockMetaDataL.getRowCount(); - while (writeLeft > 0) { - // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); - BlockMetaData blockMetaDataR = metaR.getBlocks().get(blockIdR); - List chunksR = blockMetaDataR.getColumns(); - long leftInBlock = blockMetaDataR.getRowCount() - writtenInBlock; - long writeFromBlock = Math.min(writeLeft, leftInBlock); - for (ColumnChunkMetaData chunkR : chunksR) { - // If a column is encrypted we simply throw exception. // TODO add that detail to docs - if (chunkR.isEncrypted()) { - throw new IOException("Column " + chunkR.getPath().toDotString() + " is encrypted"); - } - // This column has been pruned. - ColumnDescriptor descriptorR = descriptorsMapR.get(chunkR.getPath()); - TransParquetFileReader readerR = inputFilesR.peek(); - if (!colReadersR.containsKey(descriptorR)) { - PageReadStore pageReadStore = readerR.readRowGroup(blockIdR); - ColumnReadStoreImpl crStore = - new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, originalCreatedBy); - ColumnReader cReader = crStore.getColumnReader(descriptorR); - colReadersR.put(descriptorR, cReader); - } - ColumnReader colReaderR = colReadersR.get(descriptorR); - buildChunks(descriptorR, chunkR, colReaderR, writer, schemaR, writeFromBlock, numBlocksRewritten); - } - writeLeft -= Math.min(writeLeft, blockMetaDataR.getRowCount()); // TODO add exception for empty right schema so we don't fall with exception here? - writtenInBlock += writeFromBlock; - if (writeLeft > 0) { - blockIdR++; - } - if (blockIdR == metaR.getBlocks().size()) { - inputFilesR.poll(); - blockIdR = 0; - writtenInBlock = 0; - } + // Writing the right side + for (RightColumnWriter writer: columnWritersR) { + writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); } - writer.endBlock(); - numBlocksRewritten++; + rowGroupIdx++; } } } - public BytesInput readBlock(int length, TransParquetFileReader reader) throws IOException { - byte[] data; - if (length > pageBufferSize) { - data = new byte[length]; - } else { - data = pageBuffer; + private static class RightColumnWriter { + private final Queue inputFiles; + private final ParquetFileWriter writer; + private final MessageType schema; + private final Map descriptorsMapR; + private final Map colReadersR = new HashMap<>(); + private int rowGroupIdxR = 0; + private int writtenFromBlock = 0; + + public RightColumnWriter(Queue inputFiles, ParquetFileWriter writer) throws IOException { + this.inputFiles = inputFiles; + this.writer = writer; + this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); + this.descriptorsMapR = + this.schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + initColumnReaders(); } - reader.blockRead(data, 0, length); - return BytesInput.from(data, 0, length); - } - private void buildChunks( - ColumnDescriptor descriptor, - ColumnChunkMetaData chunk, - ColumnReader cReader, - ParquetFileWriter writer, - MessageType schema, - long rowsToWrite, - int numBlocksRewritten) - throws IOException { - - int dMax = descriptor.getMaxDefinitionLevel(); - ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = - ParquetProperties.builder().withWriterVersion(writerVersion).build(); - CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); - - // Create new schema that only has the current column - MessageType newSchema = newSchema(schema, descriptor); - ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( - compressor, - newSchema, - props.getAllocator(), - props.getColumnIndexTruncateLength(), - props.getPageWriteChecksumEnabled(), - writer.getEncryptor(), - numBlocksRewritten); - ColumnWriteStore cStore = props.newColumnWriteStore(newSchema, cPageStore); - ColumnWriter cWriter = cStore.getColumnWriter(descriptor); - Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; - - int rowCount = 0; - while (rowCount < rowsToWrite) { -// for (int i = 0; i < rowsToWrite; i++) { - int rlvl = cReader.getCurrentRepetitionLevel(); - int dlvl = cReader.getCurrentDefinitionLevel(); - if (rlvl == 0) { - if (rowCount > 0) { - cStore.endRecord(); + public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { +// LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + while (rowsToWrite > 0) { + List blocks = inputFiles.peek().getFooter().getBlocks(); + BlockMetaData block = blocks.get(rowGroupIdxR); + List chunksR = block.getColumns(); + long leftInBlock = block.getRowCount() - writtenFromBlock; + long writeFromBlock = Math.min(rowsToWrite, leftInBlock); + for (ColumnChunkMetaData chunkR : chunksR) { + if (chunkR.isEncrypted()) { + throw new IOException("Column " + chunkR.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs + } + ColumnDescriptor descriptorR = descriptorsMapR.get(chunkR.getPath()); + ColumnReader colReaderR = colReadersR.get(descriptorR); + MessageType columnSchema = newSchema(schema, descriptorR); + writeRows(colReaderR, writer, descriptorR, chunkR, columnSchema, writeFromBlock, rowGroupIdx); } - rowCount++; + rowsToWrite -= writeFromBlock; + writtenFromBlock += writeFromBlock; + if (rowsToWrite > 0) { + rowGroupIdxR++; + } + if (rowGroupIdxR == blocks.size()) { + inputFiles.poll(); + rowGroupIdxR = 0; + writtenFromBlock = 0; + } + initColumnReaders(); } - if (dlvl < dMax) { - cWriter.writeNull(rlvl, dlvl); - } else if (columnType == Integer.TYPE) { - cWriter.write(cReader.getInteger(), rlvl, dlvl); - } else if (columnType == Long.TYPE) { - cWriter.write(cReader.getLong(), rlvl, dlvl); - } else if (columnType == Float.TYPE) { - cWriter.write(cReader.getFloat(), rlvl, dlvl); - } else if (columnType == Double.TYPE) { - cWriter.write(cReader.getDouble(), rlvl, dlvl); - } else if (columnType == Binary.class) { - cWriter.write(cReader.getBinary(), rlvl, dlvl); - } else if (columnType == Boolean.TYPE) { - cWriter.write(cReader.getBoolean(), rlvl, dlvl); - } else { - throw new UnsupportedOperationException( - String.format("Unsupported column java class: %s", columnType.toString())); + } + + private void initColumnReaders() throws IOException { + if (!inputFiles.isEmpty()) { + for (ColumnDescriptor descriptorR : descriptorsMapR.values()) { + TransParquetFileReader readerR = inputFiles.peek(); + PageReadStore pageReadStore = readerR.readRowGroup(rowGroupIdxR); + String createdBy = readerR.getFooter().getFileMetaData().getCreatedBy(); + ColumnReadStoreImpl crStore = + new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, createdBy); + ColumnReader cReader = crStore.getColumnReader(descriptorR); + colReadersR.put(descriptorR, cReader); + } } - cReader.consume(); } - cStore.endRecord(); - cStore.flush(); - cPageStore.flushToFileWriter(writer); + private void writeRows( + ColumnReader reader, + ParquetFileWriter writer, + ColumnDescriptor descriptor, + ColumnChunkMetaData chunk, + MessageType columnSchema, + long rowsToWrite, + int rowGroupIdx) + throws IOException { + + int dMax = descriptor.getMaxDefinitionLevel(); + ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = + ParquetProperties.builder().withWriterVersion(writerVersion).build(); + CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); + CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + + // Create new schema that only has the current column + ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( + compressor, + columnSchema, + props.getAllocator(), + props.getColumnIndexTruncateLength(), + props.getPageWriteChecksumEnabled(), + writer.getEncryptor(), + rowGroupIdx); + ColumnWriteStore cStore = props.newColumnWriteStore(columnSchema, cPageStore); + ColumnWriter cWriter = cStore.getColumnWriter(descriptor); + Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; + + int rowCount = 0; + while (rowCount < rowsToWrite) { + int rlvl = reader.getCurrentRepetitionLevel(); + int dlvl = reader.getCurrentDefinitionLevel(); + if (rlvl == 0) { + if (rowCount > 0) { + cStore.endRecord(); + } + rowCount++; + } + if (dlvl < dMax) { + cWriter.writeNull(rlvl, dlvl); + } else if (columnType == Integer.TYPE) { + cWriter.write(reader.getInteger(), rlvl, dlvl); + } else if (columnType == Long.TYPE) { + cWriter.write(reader.getLong(), rlvl, dlvl); + } else if (columnType == Float.TYPE) { + cWriter.write(reader.getFloat(), rlvl, dlvl); + } else if (columnType == Double.TYPE) { + cWriter.write(reader.getDouble(), rlvl, dlvl); + } else if (columnType == Binary.class) { + cWriter.write(reader.getBinary(), rlvl, dlvl); + } else if (columnType == Boolean.TYPE) { + cWriter.write(reader.getBoolean(), rlvl, dlvl); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported column java class: %s", columnType.toString())); + } + reader.consume(); + } + cStore.endRecord(); - cStore.close(); - cWriter.close(); - } + cStore.flush(); + cPageStore.flushToFileWriter(writer); - private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) { - String[] path = descriptor.getPath(); - Type type = schema.getType(path); - if (path.length == 1) { - return new MessageType(schema.getName(), type); + cStore.close(); + cWriter.close(); } - for (Type field : schema.getFields()) { - if (!field.isPrimitive()) { - Type newType = extractField(field.asGroupType(), type); - if (newType != null) { - return new MessageType(schema.getName(), newType); - } + private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) { + String[] path = descriptor.getPath(); + Type type = schema.getType(path); + if (path.length == 1) { + return new MessageType(schema.getName(), type); } - } - // We should never hit this because 'type' is returned by schema.getType(). - throw new RuntimeException("No field is found"); - } + for (Type field : schema.getFields()) { + if (!field.isPrimitive()) { + Type newType = extractField(field.asGroupType(), type); + if (newType != null) { + return new MessageType(schema.getName(), newType); + } + } + } - private Type extractField(GroupType candidate, Type targetField) { - if (targetField.equals(candidate)) { - return targetField; + // We should never hit this because 'type' is returned by schema.getType(). + throw new RuntimeException("No field is found"); } - // In case 'type' is a descendants of candidate - for (Type field : candidate.asGroupType().getFields()) { - if (field.isPrimitive()) { - if (field.equals(targetField)) { - return new GroupType(candidate.getRepetition(), candidate.getName(), targetField); - } - } else { - Type tempField = extractField(field.asGroupType(), targetField); - if (tempField != null) { - return new GroupType(candidate.getRepetition(), candidate.getName(), tempField); + private Type extractField(GroupType candidate, Type targetField) { + if (targetField.equals(candidate)) { + return targetField; + } + + // In case 'type' is a descendants of candidate + for (Type field : candidate.asGroupType().getFields()) { + if (field.isPrimitive()) { + if (field.equals(targetField)) { + return new GroupType(candidate.getRepetition(), candidate.getName(), targetField); + } + } else { + Type tempField = extractField(field.asGroupType(), targetField); + if (tempField != null) { + return new GroupType(candidate.getRepetition(), candidate.getName(), tempField); + } } } - } - return null; - } + return null; + } - private static final class DummyGroupConverter extends GroupConverter { - @Override - public void start() {} + private static final class DummyGroupConverter extends GroupConverter { + @Override + public void start() {} - @Override - public void end() {} + @Override + public void end() {} - @Override - public Converter getConverter(int fieldIndex) { - return new DummyConverter(); + @Override + public Converter getConverter(int fieldIndex) { + return new DummyConverter(); + } } - } - private static final class DummyConverter extends PrimitiveConverter { - @Override - public GroupConverter asGroupConverter() { - return new DummyGroupConverter(); + private static final class DummyConverter extends PrimitiveConverter { + @Override + public GroupConverter asGroupConverter() { + return new DummyGroupConverter(); + } } + } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java index e6c7226bd4..5305c910ef 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java @@ -18,54 +18,43 @@ */ package org.apache.parquet.hadoop.join; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; +import static org.apache.parquet.schema.Type.Repetition.*; +import static org.junit.Assert.*; + +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.Version; import org.apache.parquet.column.ParquetProperties; -import org.apache.parquet.column.values.bloomfilter.BloomFilter; import org.apache.parquet.conf.ParquetConfiguration; import org.apache.parquet.conf.PlainParquetConfiguration; import org.apache.parquet.crypto.FileDecryptionProperties; -import org.apache.parquet.crypto.FileEncryptionProperties; -import org.apache.parquet.crypto.ParquetCipher; -import org.apache.parquet.crypto.ParquetCryptoRuntimeException; 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.IndexCache; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; -import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.GroupReadSupport; -import org.apache.parquet.hadoop.metadata.*; -import org.apache.parquet.hadoop.rewrite.MaskMode; -import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; -import org.apache.parquet.hadoop.util.*; -import org.apache.parquet.internal.column.columnindex.ColumnIndex; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.EncryptionTestFile; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.hadoop.util.TestFileBuilder; import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.InvalidRecordException; import org.apache.parquet.io.OutputFile; -import org.apache.parquet.io.SeekableInputStream; -import org.apache.parquet.schema.*; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; -import java.util.*; -import java.util.stream.Collectors; - -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; -import static org.apache.parquet.schema.Type.Repetition.*; -import static org.junit.Assert.*; @RunWith(Parameterized.class) public class ParquetJoinTest { @@ -120,7 +109,7 @@ public void testMergeTwoFilesOnly() throws Exception { for (EncryptionTestFile inputFile : inputFilesR) { inputPathsR.add(new Path(inputFile.getFileName())); } - JoinOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); + JoinOptions.Builder builder = createBuilder(inputPathsL, ImmutableList.of(inputPathsR)); JoinOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); joiner = new ParquetJoiner(options); @@ -225,9 +214,7 @@ private void validateColumnData( } - - - private JoinOptions.Builder createBuilder(List inputPathsL, List inputPathsR) throws IOException { + private JoinOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { JoinOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); @@ -237,8 +224,9 @@ private JoinOptions.Builder createBuilder(List inputPathsL, List inp List inputsL = inputPathsL.stream() .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) .collect(Collectors.toList()); - List inputsR = inputPathsR.stream() - .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) + List> inputsR = inputPathsR + .stream() + .map(x -> x.stream().map(y -> (InputFile) HadoopInputFile.fromPathUnchecked(y, conf)).collect(Collectors.toList())) .collect(Collectors.toList()); builder = new JoinOptions.Builder(parquetConf, inputsL, inputsR, outputPath); } From 05eb22a28d43a32a8b15b4c64c54f31da0a98354 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 19 Feb 2024 18:51:57 -0800 Subject: [PATCH 04/57] extend the main test for multiple files on the right --- .../parquet/hadoop/join/ParquetJoiner.java | 17 ++- .../parquet/hadoop/join/ParquetJoinTest.java | 120 ++++++++++++------ 2 files changed, 92 insertions(+), 45 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java index cd492e5e2e..6cdec21d9d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java @@ -100,8 +100,10 @@ public ParquetJoiner(JoinOptions options) throws IOException { .collect(Collectors.toList()); // TODO check that there is no overlap of fields on the right - Map fieldNamesL = schemaL.getFields().stream().collect(Collectors.toMap(Type::getName, x -> x)); - Map fieldNamesR = schemaR.stream().flatMap(x -> x.getFields().stream()).collect(Collectors.toMap(Type::getName, x -> x)); + Map fieldNamesL = new LinkedHashMap<>(); + schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); + Map fieldNamesR = new LinkedHashMap<>(); + schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); List fields = Stream.concat( fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left @@ -115,10 +117,13 @@ public ParquetJoiner(JoinOptions options) throws IOException { this.indexCacheStrategy = options.getIndexCacheStrategy(); long rowCountL = inputFilesL.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - long rowCountR = inputFilesR.stream().flatMap(Collection::stream).mapToLong(ParquetFileReader::getRecordCount).sum(); - if (rowCountL != rowCountR) { - throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); - } + inputFilesR.stream() + .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) + .forEach(rowCountR -> { + if (rowCountL != rowCountR) { + throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); + } + }); ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; this.writer = new ParquetFileWriter( diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java index 5305c910ef..5a81f36747 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java @@ -67,20 +67,20 @@ public class ParquetJoinTest { private final boolean usingHadoop; private List inputFilesL = null; - private List inputFilesR = null; + private List> inputFilesR = null; private String outputFile = null; private ParquetJoiner joiner = null; @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { return new Object[][] { -// {"v1", "NONE", true}, -// {"v1", "PREFETCH_BLOCK", true}, -// {"v2", "NONE", true}, -// {"v2", "PREFETCH_BLOCK", true}, -// {"v1", "NONE", false}, -// {"v1", "PREFETCH_BLOCK", false}, -// {"v2", "NONE", false}, + {"v1", "NONE", true}, + {"v1", "PREFETCH_BLOCK", true}, + {"v2", "NONE", true}, + {"v2", "PREFETCH_BLOCK", true}, + {"v1", "NONE", false}, + {"v1", "PREFETCH_BLOCK", false}, + {"v2", "NONE", false}, {"v2", "PREFETCH_BLOCK", false} }; } @@ -98,18 +98,16 @@ public ParquetJoinTest(String writerVersion, String indexCacheStrategy, boolean @Test public void testMergeTwoFilesOnly() throws Exception { - testMultipleInputFilesSetup(); + testSingleInputFileSetup(); // Only merge two files but do not change anything. - List inputPathsL = new ArrayList<>(); - for (EncryptionTestFile inputFile : inputFilesL) { - inputPathsL.add(new Path(inputFile.getFileName())); - } - List inputPathsR = new ArrayList<>(); - for (EncryptionTestFile inputFile : inputFilesR) { - inputPathsR.add(new Path(inputFile.getFileName())); - } - JoinOptions.Builder builder = createBuilder(inputPathsL, ImmutableList.of(inputPathsR)); + List inputPathsL = inputFilesL.stream() + .map(x -> new Path(x.getFileName())) + .collect(Collectors.toList()); + List> inputPathsR = inputFilesR.stream() + .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) + .collect(Collectors.toList()); + JoinOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); JoinOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); joiner = new ParquetJoiner(options); @@ -120,44 +118,71 @@ public void testMergeTwoFilesOnly() throws Exception { ParquetMetadata pmd = ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); MessageType schema = pmd.getFileMetaData().getSchema(); - MessageType expectSchema = createSchemaL(); -// assertEquals(expectSchema, schema); + MessageType expectSchema = createSchema(); + assertEquals(expectSchema, schema); // Verify the merged data are not changed validateColumnData(null); } - private void testMultipleInputFilesSetup() throws IOException { - inputFilesL = Lists.newArrayList(); - inputFilesL.add(new TestFileBuilder(conf, createSchemaL()) + private void testSingleInputFileSetup() throws IOException { + inputFilesL = Lists.newArrayList( + new TestFileBuilder(conf, createSchemaL()) .withNumRecord(numRecord) .withCodec("GZIP") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) - .build()); - inputFilesR = Lists.newArrayList(); - inputFilesR.add(new TestFileBuilder(conf, createSchemaR()) - .withNumRecord(numRecord) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build()); + .build() + ); + inputFilesR = Lists.newArrayList( + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR1()) + .withNumRecord(numRecord) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ), + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ) + ); } - private MessageType createSchemaL() { + private MessageType createSchema() { return new MessageType( "schema", new PrimitiveType(OPTIONAL, INT64, "DocId"), new PrimitiveType(REQUIRED, BINARY, "Name"), new PrimitiveType(OPTIONAL, BINARY, "Gender"), - new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), + new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction"), + new GroupType( + OPTIONAL, + "Links", + new PrimitiveType(REPEATED, BINARY, "Backward"), + new PrimitiveType(REPEATED, BINARY, "Forward"))); } + private MessageType createSchemaL() { + return new MessageType( + "schema", + new PrimitiveType(OPTIONAL, INT64, "DocId"), + new PrimitiveType(REQUIRED, BINARY, "Name"), + new PrimitiveType(OPTIONAL, BINARY, "Gender"), + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), + new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); + } private MessageType createSchemaR() { return new MessageType( "schema", - new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction"), + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), new GroupType( OPTIONAL, "Links", @@ -165,6 +190,22 @@ private MessageType createSchemaR() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } + private MessageType createSchemaR1() { + return new MessageType( + "schema", + new GroupType( + OPTIONAL, + "Links", + new PrimitiveType(REPEATED, BINARY, "Backward"), + new PrimitiveType(REPEATED, BINARY, "Forward"))); + } + + private MessageType createSchemaR2() { + return new MessageType( + "schema", + new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); + } + private void validateColumnData( FileDecryptionProperties fileDecryptionProperties) throws IOException { @@ -184,7 +225,8 @@ private void validateColumnData( assertNotNull(group); SimpleGroup expectGroupL = inputFilesL.get(i / numRecord).getFileContent()[i % numRecord]; - SimpleGroup expectGroupR = inputFilesR.get(i / numRecord).getFileContent()[i % numRecord]; + SimpleGroup expectGroupR1 = inputFilesR.get(0).get(i / numRecord).getFileContent()[i % numRecord]; + SimpleGroup expectGroupR2 = inputFilesR.get(1).get(i / numRecord).getFileContent()[i % numRecord]; assertEquals(group.getLong("DocId", 0), expectGroupL.getLong("DocId", 0)); assertArrayEquals( @@ -193,18 +235,18 @@ private void validateColumnData( assertArrayEquals( group.getBinary("Gender", 0).getBytes(), expectGroupL.getBinary("Gender", 0).getBytes()); - assertEquals(group.getFloat("FloatFraction", 0), expectGroupL.getFloat("FloatFraction", 0), 0); - assertEquals(group.getDouble("DoubleFraction", 0), expectGroupR.getDouble("DoubleFraction", 0), 0); + assertEquals(group.getFloat("FloatFraction", 0), expectGroupR2.getFloat("FloatFraction", 0), 0); + assertEquals(group.getDouble("DoubleFraction", 0), expectGroupL.getDouble("DoubleFraction", 0), 0); Group subGroup = group.getGroup("Links", 0); assertArrayEquals( subGroup.getBinary("Backward", 0).getBytes(), - expectGroupR + expectGroupR1 .getGroup("Links", 0) .getBinary("Backward", 0) .getBytes()); assertArrayEquals( subGroup.getBinary("Forward", 0).getBytes(), - expectGroupR + expectGroupR1 .getGroup("Links", 0) .getBinary("Forward", 0) .getBytes()); From 6bb950d38bf7c00b19b6cdd0ecc5f1f5d949d24d Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 21 Feb 2024 18:11:03 -0800 Subject: [PATCH 05/57] extend the main test for multiple files on the right --- .../parquet/hadoop/join/ParquetJoiner.java | 209 ++++++++++-------- .../parquet/hadoop/join/ParquetJoinTest.java | 91 +++++--- 2 files changed, 173 insertions(+), 127 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java index 6cdec21d9d..29d4562aff 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java @@ -228,134 +228,157 @@ private static class RightColumnWriter { private final Queue inputFiles; private final ParquetFileWriter writer; private final MessageType schema; - private final Map descriptorsMapR; - private final Map colReadersR = new HashMap<>(); - private int rowGroupIdxR = 0; + private final Map descriptorsMap; + private final Map colReaders = new HashMap<>(); + private final Map cPageStores = new HashMap<>(); + private final Map cStores = new HashMap<>(); + private final Map cWriters = new HashMap<>(); + private int rowGroupIdxIn = 0; + private int rowGroupIdxOut = 0; private int writtenFromBlock = 0; public RightColumnWriter(Queue inputFiles, ParquetFileWriter writer) throws IOException { this.inputFiles = inputFiles; this.writer = writer; this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - this.descriptorsMapR = + this.descriptorsMap = this.schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - initColumnReaders(); + initReaders(); + initWriters(); } public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { -// LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + if (rowGroupIdxIn != rowGroupIdx) { + this.rowGroupIdxIn = rowGroupIdx; + flushWriters(); + initWriters(); + } while (rowsToWrite > 0) { List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(rowGroupIdxR); - List chunksR = block.getColumns(); + BlockMetaData block = blocks.get(this.rowGroupIdxOut); + List chunks = block.getColumns(); long leftInBlock = block.getRowCount() - writtenFromBlock; long writeFromBlock = Math.min(rowsToWrite, leftInBlock); - for (ColumnChunkMetaData chunkR : chunksR) { - if (chunkR.isEncrypted()) { - throw new IOException("Column " + chunkR.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs + for (ColumnChunkMetaData chunk : chunks) { + if (chunk.isEncrypted()) { // TODO check this during construction? + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs } - ColumnDescriptor descriptorR = descriptorsMapR.get(chunkR.getPath()); - ColumnReader colReaderR = colReadersR.get(descriptorR); - MessageType columnSchema = newSchema(schema, descriptorR); - writeRows(colReaderR, writer, descriptorR, chunkR, columnSchema, writeFromBlock, rowGroupIdx); + ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); + copyValues(descriptor, writeFromBlock); } rowsToWrite -= writeFromBlock; writtenFromBlock += writeFromBlock; - if (rowsToWrite > 0) { - rowGroupIdxR++; - } - if (rowGroupIdxR == blocks.size()) { - inputFiles.poll(); - rowGroupIdxR = 0; + if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { + this.rowGroupIdxOut++; + if (this.rowGroupIdxOut == blocks.size()) { + inputFiles.poll(); + this.rowGroupIdxOut = 0; + } writtenFromBlock = 0; + // this is called after all rows are processed + initReaders(); } - initColumnReaders(); } + flushWriters(); + } + + private void flushWriters() throws IOException { + cStores.values().forEach(cStore -> { + cStore.flush(); + cStore.close(); + }); + cWriters.values().forEach(ColumnWriter::close); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + if (cPageStores.containsKey(descriptor)) + cPageStores.get(descriptor).flushToFileWriter(writer); + } + cStores.clear(); + cWriters.clear(); + cPageStores.clear(); } - private void initColumnReaders() throws IOException { + private void initWriters() { if (!inputFiles.isEmpty()) { - for (ColumnDescriptor descriptorR : descriptorsMapR.values()) { - TransParquetFileReader readerR = inputFiles.peek(); - PageReadStore pageReadStore = readerR.readRowGroup(rowGroupIdxR); - String createdBy = readerR.getFooter().getFileMetaData().getCreatedBy(); - ColumnReadStoreImpl crStore = - new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, createdBy); - ColumnReader cReader = crStore.getColumnReader(descriptorR); - colReadersR.put(descriptorR, cReader); + List blocks = inputFiles.peek().getFooter().getBlocks(); + BlockMetaData block = blocks.get(this.rowGroupIdxOut); + ColumnChunkMetaData chunk = block.getColumns().get(0); // TODO use to current chunk idx? + ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = + ParquetProperties.builder().withWriterVersion(writerVersion).build(); + CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); + CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + MessageType columnSchema = newSchema(schema, descriptor); + ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( + compressor, + columnSchema, + props.getAllocator(), + props.getColumnIndexTruncateLength(), + props.getPageWriteChecksumEnabled(), + writer.getEncryptor(), + rowGroupIdxIn); + ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore); + ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); + cPageStores.put(descriptor, cPageStore); + cStores.put(descriptor, cwStore); + cWriters.put(descriptor, cWriter); } } } - private void writeRows( - ColumnReader reader, - ParquetFileWriter writer, - ColumnDescriptor descriptor, - ColumnChunkMetaData chunk, - MessageType columnSchema, - long rowsToWrite, - int rowGroupIdx) - throws IOException { + private void initReaders() throws IOException { + if (!inputFiles.isEmpty()) { + TransParquetFileReader reader = inputFiles.peek(); + PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxOut); + String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); + ColumnReadStoreImpl crStore = + new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, createdBy); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + ColumnReader cReader = crStore.getColumnReader(descriptor); + colReaders.put(descriptor, cReader); + } + } + } + private void copyValues( + ColumnDescriptor descriptor, + long rowsToWrite) { + ColumnWriteStore cStore = cStores.get(descriptor); + ColumnWriter cWriter = cWriters.get(descriptor); int dMax = descriptor.getMaxDefinitionLevel(); - ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = - ParquetProperties.builder().withWriterVersion(writerVersion).build(); - CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); - - // Create new schema that only has the current column - ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( - compressor, - columnSchema, - props.getAllocator(), - props.getColumnIndexTruncateLength(), - props.getPageWriteChecksumEnabled(), - writer.getEncryptor(), - rowGroupIdx); - ColumnWriteStore cStore = props.newColumnWriteStore(columnSchema, cPageStore); - ColumnWriter cWriter = cStore.getColumnWriter(descriptor); Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; - - int rowCount = 0; - while (rowCount < rowsToWrite) { + ColumnReader reader = colReaders.get(descriptor); + for (int i = 0; i < rowsToWrite; i++) { int rlvl = reader.getCurrentRepetitionLevel(); int dlvl = reader.getCurrentDefinitionLevel(); - if (rlvl == 0) { - if (rowCount > 0) { - cStore.endRecord(); + do { + if (dlvl < dMax) { + cWriter.writeNull(rlvl, dlvl); + } else if (columnType == Integer.TYPE) { + cWriter.write(reader.getInteger(), rlvl, dlvl); + } else if (columnType == Long.TYPE) { + cWriter.write(reader.getLong(), rlvl, dlvl); + } else if (columnType == Float.TYPE) { + cWriter.write(reader.getFloat(), rlvl, dlvl); + } else if (columnType == Double.TYPE) { + cWriter.write(reader.getDouble(), rlvl, dlvl); + } else if (columnType == Binary.class) { + cWriter.write(reader.getBinary(), rlvl, dlvl); + } else if (columnType == Boolean.TYPE) { + cWriter.write(reader.getBoolean(), rlvl, dlvl); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported column java class: %s", columnType.toString())); } - rowCount++; - } - if (dlvl < dMax) { - cWriter.writeNull(rlvl, dlvl); - } else if (columnType == Integer.TYPE) { - cWriter.write(reader.getInteger(), rlvl, dlvl); - } else if (columnType == Long.TYPE) { - cWriter.write(reader.getLong(), rlvl, dlvl); - } else if (columnType == Float.TYPE) { - cWriter.write(reader.getFloat(), rlvl, dlvl); - } else if (columnType == Double.TYPE) { - cWriter.write(reader.getDouble(), rlvl, dlvl); - } else if (columnType == Binary.class) { - cWriter.write(reader.getBinary(), rlvl, dlvl); - } else if (columnType == Boolean.TYPE) { - cWriter.write(reader.getBoolean(), rlvl, dlvl); - } else { - throw new UnsupportedOperationException( - String.format("Unsupported column java class: %s", columnType.toString())); - } - reader.consume(); + reader.consume(); + rlvl = reader.getCurrentRepetitionLevel(); + dlvl = reader.getCurrentDefinitionLevel(); + } while (rlvl > 0); + cStore.endRecord(); } - cStore.endRecord(); - - cStore.flush(); - cPageStore.flushToFileWriter(writer); - - cStore.close(); - cWriter.close(); } private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) { diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java index 5a81f36747..64e5ddcad6 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java @@ -22,7 +22,6 @@ import static org.apache.parquet.schema.Type.Repetition.*; import static org.junit.Assert.*; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; import java.util.ArrayList; @@ -59,7 +58,7 @@ @RunWith(Parameterized.class) public class ParquetJoinTest { - private final int numRecord = 100000; + private final int numRecord = 100_000; private final Configuration conf = new Configuration(); private final ParquetConfiguration parquetConf = new PlainParquetConfiguration(); private final ParquetProperties.WriterVersion writerVersion; @@ -98,7 +97,7 @@ public ParquetJoinTest(String writerVersion, String indexCacheStrategy, boolean @Test public void testMergeTwoFilesOnly() throws Exception { - testSingleInputFileSetup(); + testMultiInputFileSetup(); // Only merge two files but do not change anything. List inputPathsL = inputFilesL.stream() @@ -125,10 +124,18 @@ public void testMergeTwoFilesOnly() throws Exception { validateColumnData(null); } - private void testSingleInputFileSetup() throws IOException { + private void testMultiInputFileSetup() throws IOException { inputFilesL = Lists.newArrayList( new TestFileBuilder(conf, createSchemaL()) - .withNumRecord(numRecord) + .withNumRecord(numRecord / 2) + .withRowGroupSize(5_000_000) + .withCodec("GZIP") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaL()) + .withNumRecord(numRecord - (numRecord / 2)) + .withRowGroupSize(6_000_000) .withCodec("GZIP") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) @@ -136,20 +143,38 @@ private void testSingleInputFileSetup() throws IOException { ); inputFilesR = Lists.newArrayList( Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR1()) - .withNumRecord(numRecord) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ), - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR1()) + .withNumRecord(numRecord) + .withRowGroupSize(7_000_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ), + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(200_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(300_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord - 2 * (numRecord / 3)) + .withRowGroupSize(400_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ) ) ); } @@ -179,17 +204,6 @@ private MessageType createSchemaL() { new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); } - private MessageType createSchemaR() { - return new MessageType( - "schema", - new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), - new GroupType( - OPTIONAL, - "Links", - new PrimitiveType(REPEATED, BINARY, "Backward"), - new PrimitiveType(REPEATED, BINARY, "Forward"))); - } - private MessageType createSchemaR1() { return new MessageType( "schema", @@ -220,13 +234,22 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } + int idxFileL = 0; + int idxFileR1 = 0; + int idxFileR2 = 0; + int idxRowL = 0; + int idxRowR1 = 0; + int idxRowR2 = 0; for (int i = 0; i < totalRows; i++) { Group group = reader.read(); assertNotNull(group); - SimpleGroup expectGroupL = inputFilesL.get(i / numRecord).getFileContent()[i % numRecord]; - SimpleGroup expectGroupR1 = inputFilesR.get(0).get(i / numRecord).getFileContent()[i % numRecord]; - SimpleGroup expectGroupR2 = inputFilesR.get(1).get(i / numRecord).getFileContent()[i % numRecord]; + if (idxRowL >= inputFilesL.get(idxFileL).getFileContent().length) { idxFileL++; idxRowL = 0; } + if (idxRowR1 >= inputFilesR.get(0).get(idxFileR1).getFileContent().length) { idxFileR1++; idxRowR1 = 0; } + if (idxRowR2 >= inputFilesR.get(1).get(idxFileR2).getFileContent().length) { idxFileR2++; idxRowR2 = 0; } + SimpleGroup expectGroupL = inputFilesL.get(idxFileL).getFileContent()[idxRowL++]; + SimpleGroup expectGroupR1 = inputFilesR.get(0).get(idxFileR1).getFileContent()[idxRowR1++]; + SimpleGroup expectGroupR2 = inputFilesR.get(1).get(idxFileR2).getFileContent()[idxRowR2++]; assertEquals(group.getLong("DocId", 0), expectGroupL.getLong("DocId", 0)); assertArrayEquals( @@ -235,7 +258,7 @@ private void validateColumnData( assertArrayEquals( group.getBinary("Gender", 0).getBytes(), expectGroupL.getBinary("Gender", 0).getBytes()); - assertEquals(group.getFloat("FloatFraction", 0), expectGroupR2.getFloat("FloatFraction", 0), 0); + assertEquals(expectGroupR2.getFloat("FloatFraction", 0), expectGroupR2.getFloat("FloatFraction", 0), 0); assertEquals(group.getDouble("DoubleFraction", 0), expectGroupL.getDouble("DoubleFraction", 0), 0); Group subGroup = group.getGroup("Links", 0); assertArrayEquals( From f9536c355cb2bb7617e8a89371294c5246450241 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 22 Feb 2024 18:57:13 -0800 Subject: [PATCH 06/57] converge join logic, crate a draft of options and rewriter --- .../parquet/hadoop/join/JoinOptions.java | 154 ------ .../parquet/hadoop/join/ParquetJoiner.java | 448 ------------------ .../hadoop/rewrite/ParquetRewriter.java | 338 ++++++++++++- .../hadoop/rewrite/RewriteOptions.java | 85 +++- .../parquet/hadoop/join/ParquetJoinTest.java | 26 +- 5 files changed, 421 insertions(+), 630 deletions(-) delete mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java delete mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java deleted file mode 100644 index 18433fcd4b..0000000000 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/JoinOptions.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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.join; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.Preconditions; -import org.apache.parquet.conf.HadoopParquetConfiguration; -import org.apache.parquet.conf.ParquetConfiguration; -import org.apache.parquet.hadoop.IndexCache; -import org.apache.parquet.hadoop.util.ConfigurationUtil; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.hadoop.util.HadoopOutputFile; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.OutputFile; -import java.util.List; -import java.util.stream.Collectors; - -public class JoinOptions { - - private final ParquetConfiguration conf; - private final List inputFilesL; - private final List> inputFilesR; - private final OutputFile outputFile; - private final IndexCache.CacheStrategy indexCacheStrategy; - - private JoinOptions( - ParquetConfiguration conf, - List inputFilesL, - List> inputFilesR, - OutputFile outputFile, - IndexCache.CacheStrategy indexCacheStrategy) { - this.conf = conf; - this.inputFilesL = inputFilesL; - this.inputFilesR = inputFilesR; - this.outputFile = outputFile; - this.indexCacheStrategy = indexCacheStrategy; - } - - public Configuration getConf() { - return ConfigurationUtil.createHadoopConfiguration(conf); - } - - public ParquetConfiguration getParquetConfiguration() { - return conf; - } - - private List getInputFiles(List in) { - return in.stream() - .map(f -> { - if (f instanceof HadoopOutputFile) { - HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) f; - return new Path(hadoopOutputFile.getPath()); - } else { - throw new RuntimeException("The input files do not all have an associated Hadoop Path."); - } - }) - .collect(Collectors.toList()); - } - - public List getParquetInputFilesL() { - return inputFilesL; - } - - public List> getParquetInputFilesR() { - return inputFilesR; - } - - public Path getOutputFile() { - if (outputFile instanceof HadoopOutputFile) { - HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) outputFile; - return new Path(hadoopOutputFile.getPath()); - } else { - throw new RuntimeException("The output file does not have an associated Hadoop Path."); - } - } - - public OutputFile getParquetOutputFile() { - return outputFile; - } - - public IndexCache.CacheStrategy getIndexCacheStrategy() { - return indexCacheStrategy; - } - - public static class Builder { - private final ParquetConfiguration conf; - private final List inputFilesL; - private final List> inputFilesR; - private final OutputFile outputFile; - private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; - - public Builder(Configuration conf, List inputFilesL, List> inputFilesR, Path outputFile) { - this.conf = new HadoopParquetConfiguration(conf); - this.inputFilesL = inputFilesL - .stream() - .map(x -> HadoopInputFile.fromPathUnchecked(x, conf)) - .collect(Collectors.toList()); - this.inputFilesR = inputFilesR - .stream() - .map(x -> x.stream().map(y -> (InputFile) HadoopInputFile.fromPathUnchecked(y, conf)).collect(Collectors.toList())) - .collect(Collectors.toList()); - this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); - } - - public Builder( - ParquetConfiguration conf, - List inputFilesL, - List> inputFilesR, - OutputFile outputFile) { - this.conf = conf; - this.inputFilesL = inputFilesL; - this.inputFilesR = inputFilesR; - this.outputFile = outputFile; - } - - public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { - this.indexCacheStrategy = cacheStrategy; - return this; - } - - public JoinOptions build() { - // TODO move these check to the ParquetJoiner itself? - Preconditions.checkArgument(inputFilesL != null && !inputFilesL.isEmpty(), "Input fileL can't be NULL or empty"); - Preconditions.checkArgument(inputFilesR != null && !inputFilesR.isEmpty(), "Input fileR can't be NULL or empty"); - Preconditions.checkArgument(inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), "Input fileR elements can't be NULL or empty"); - Preconditions.checkArgument(outputFile != null, "Output file is required"); - - return new JoinOptions( - conf, - inputFilesL, - inputFilesR, - outputFile, - indexCacheStrategy); - } - } - -} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java deleted file mode 100644 index 29d4562aff..0000000000 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/join/ParquetJoiner.java +++ /dev/null @@ -1,448 +0,0 @@ -/* - * 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.join; - -import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; -import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.Preconditions; -import org.apache.parquet.column.*; -import org.apache.parquet.column.impl.ColumnReadStoreImpl; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.column.values.bloomfilter.BloomFilter; -import org.apache.parquet.compression.CompressionCodecFactory; -import org.apache.parquet.conf.ParquetConfiguration; -import org.apache.parquet.hadoop.*; -import org.apache.parquet.hadoop.metadata.*; -import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; -import org.apache.parquet.internal.column.columnindex.ColumnIndex; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.OutputFile; -import org.apache.parquet.io.api.Binary; -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.Type; -import org.apache.parquet.schema.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -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.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE; -import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT; - -public class ParquetJoiner implements Closeable { - - // Key to store original writer version in the file key-value metadata - public static final String ORIGINAL_CREATED_BY_KEY = "original.created.by"; - private static final Logger LOG = LoggerFactory.getLogger(ParquetJoiner.class); - - // Configurations for the new file - private final Map extraMetaData; - // Writer to rewrite the input files - private final ParquetFileWriter writer; - - // Reader and relevant states of the in-processing input file - private final Queue inputFilesL; - private final List columnWritersR = new ArrayList<>(); - private final Map descriptorsMapL; - // The index cache strategy - private final IndexCache.CacheStrategy indexCacheStrategy; - - public ParquetJoiner(JoinOptions options) throws IOException { - ParquetConfiguration conf = options.getParquetConfiguration(); - OutputFile outFile = options.getParquetOutputFile(); - this.inputFilesL = getFileReaders(options.getParquetInputFilesL(), conf); - List> inputFilesR = options.getParquetInputFilesR() - .stream() - .map(x -> getFileReaders(x, conf)) - .collect(Collectors.toList()); - ensureSameSchema(inputFilesL); - inputFilesR.forEach(this::ensureSameSchema); - LOG.info("Start rewriting {} input file(s) {} to {}", inputFilesL.size(), options.getParquetInputFilesL(), outFile); // TODO add logging for all the files - - this.extraMetaData = ImmutableMap.of( - ORIGINAL_CREATED_BY_KEY, - Stream.concat(inputFilesL.stream(), inputFilesR.stream().flatMap(Collection::stream)) - .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) - .reduce((a, b) -> a + "\n" + b) - .orElse("") - ); - - // TODO check that schema on the left and on the right is not identical - MessageType schemaL = inputFilesL.peek().getFooter().getFileMetaData().getSchema(); - List schemaR = inputFilesR - .stream() - .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) - .collect(Collectors.toList()); - - // TODO check that there is no overlap of fields on the right - Map fieldNamesL = new LinkedHashMap<>(); - schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); - Map fieldNamesR = new LinkedHashMap<>(); - schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); - List fields = Stream.concat( - fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can - fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left - ).collect(Collectors.toList()); - // Schema of input files (should be the same) and to write to the output file - MessageType schema = new MessageType(schemaL.getName(), fields); - - this.descriptorsMapL = schemaL.getColumns().stream() - .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) - .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - this.indexCacheStrategy = options.getIndexCacheStrategy(); - - long rowCountL = inputFilesL.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - inputFilesR.stream() - .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) - .forEach(rowCountR -> { - if (rowCountL != rowCountR) { - throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); - } - }); - - ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; - this.writer = new ParquetFileWriter( - outFile, - schema, - writerMode, - DEFAULT_BLOCK_SIZE, - MAX_PADDING_SIZE_DEFAULT, - DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, - DEFAULT_STATISTICS_TRUNCATE_LENGTH, - ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); - this.writer.start(); - - for (Queue inFiles: inputFilesR) { - this.columnWritersR.add(new RightColumnWriter(inFiles, writer)); - } - } - - // Open all input files to validate their schemas are compatible to merge - private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { - Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); - LinkedList inputFileReaders = new LinkedList<>(); - for (InputFile inputFile : inputFiles) { - try { - TransParquetFileReader reader = new TransParquetFileReader( - inputFile, ParquetReadOptions.builder(conf).build()); - inputFileReaders.add(reader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to open input file: " + inputFile, e); - } - } - return inputFileReaders; - } - - private void ensureSameSchema(Queue inputFileReaders) { - MessageType schema = null; - for (TransParquetFileReader reader : inputFileReaders) { - MessageType newSchema = reader.getFooter().getFileMetaData().getSchema(); - if (schema == null) { - schema = newSchema; - } else { - // Now we enforce equality of schemas from input files for simplicity. - if (!schema.equals(newSchema)) { - String file = reader.getFile(); - LOG.error( - "Input files have different schemas, expect: {}, input: {}, current file: {}", - schema, - newSchema, - file); - throw new InvalidSchemaException( - "Input files have different schemas, current file: " + file); - } - } - } - } - - @Override - public void close() throws IOException { - writer.end(extraMetaData); - } - - // TODO add the test for empty files joins, it should merge schemas - public void processBlocks() throws IOException { - int rowGroupIdx = 0; - while (!inputFilesL.isEmpty()) { - TransParquetFileReader reader = inputFilesL.poll(); - IndexCache indexCache = IndexCache.create(reader, descriptorsMapL.keySet(), indexCacheStrategy, true); - LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFilesL.size()); - List blocks = reader.getFooter().getBlocks(); - for (BlockMetaData blockMetaData: blocks) { - writer.startBlock(blockMetaData.getRowCount()); - - // Writing the left side - indexCache.setBlockMetadata(blockMetaData); - List chunksL = blockMetaData.getColumns(); - for (ColumnChunkMetaData chunk : chunksL) { - if (chunk.isEncrypted()) { // TODO add that detail to docs - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); - } - ColumnDescriptor descriptorL = descriptorsMapL.get(chunk.getPath()); - if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join - reader.setStreamPosition(chunk.getStartingPos()); - BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); - ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); - OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); - } - } - - // Writing the right side - for (RightColumnWriter writer: columnWritersR) { - writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); - } - - writer.endBlock(); - rowGroupIdx++; - } - } - } - - private static class RightColumnWriter { - private final Queue inputFiles; - private final ParquetFileWriter writer; - private final MessageType schema; - private final Map descriptorsMap; - private final Map colReaders = new HashMap<>(); - private final Map cPageStores = new HashMap<>(); - private final Map cStores = new HashMap<>(); - private final Map cWriters = new HashMap<>(); - private int rowGroupIdxIn = 0; - private int rowGroupIdxOut = 0; - private int writtenFromBlock = 0; - - public RightColumnWriter(Queue inputFiles, ParquetFileWriter writer) throws IOException { - this.inputFiles = inputFiles; - this.writer = writer; - this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - this.descriptorsMap = - this.schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - initReaders(); - initWriters(); - } - - public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { - // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); - if (rowGroupIdxIn != rowGroupIdx) { - this.rowGroupIdxIn = rowGroupIdx; - flushWriters(); - initWriters(); - } - while (rowsToWrite > 0) { - List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(this.rowGroupIdxOut); - List chunks = block.getColumns(); - long leftInBlock = block.getRowCount() - writtenFromBlock; - long writeFromBlock = Math.min(rowsToWrite, leftInBlock); - for (ColumnChunkMetaData chunk : chunks) { - if (chunk.isEncrypted()) { // TODO check this during construction? - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs - } - ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); - copyValues(descriptor, writeFromBlock); - } - rowsToWrite -= writeFromBlock; - writtenFromBlock += writeFromBlock; - if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { - this.rowGroupIdxOut++; - if (this.rowGroupIdxOut == blocks.size()) { - inputFiles.poll(); - this.rowGroupIdxOut = 0; - } - writtenFromBlock = 0; - // this is called after all rows are processed - initReaders(); - } - } - flushWriters(); - } - - private void flushWriters() throws IOException { - cStores.values().forEach(cStore -> { - cStore.flush(); - cStore.close(); - }); - cWriters.values().forEach(ColumnWriter::close); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { - if (cPageStores.containsKey(descriptor)) - cPageStores.get(descriptor).flushToFileWriter(writer); - } - cStores.clear(); - cWriters.clear(); - cPageStores.clear(); - } - - private void initWriters() { - if (!inputFiles.isEmpty()) { - List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(this.rowGroupIdxOut); - ColumnChunkMetaData chunk = block.getColumns().get(0); // TODO use to current chunk idx? - ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = - ParquetProperties.builder().withWriterVersion(writerVersion).build(); - CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { - MessageType columnSchema = newSchema(schema, descriptor); - ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( - compressor, - columnSchema, - props.getAllocator(), - props.getColumnIndexTruncateLength(), - props.getPageWriteChecksumEnabled(), - writer.getEncryptor(), - rowGroupIdxIn); - ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore); - ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); - cPageStores.put(descriptor, cPageStore); - cStores.put(descriptor, cwStore); - cWriters.put(descriptor, cWriter); - } - } - } - - private void initReaders() throws IOException { - if (!inputFiles.isEmpty()) { - TransParquetFileReader reader = inputFiles.peek(); - PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxOut); - String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); - ColumnReadStoreImpl crStore = - new ColumnReadStoreImpl(pageReadStore, new DummyGroupConverter(), schema, createdBy); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { - ColumnReader cReader = crStore.getColumnReader(descriptor); - colReaders.put(descriptor, cReader); - } - } - } - - private void copyValues( - ColumnDescriptor descriptor, - long rowsToWrite) { - ColumnWriteStore cStore = cStores.get(descriptor); - ColumnWriter cWriter = cWriters.get(descriptor); - int dMax = descriptor.getMaxDefinitionLevel(); - Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; - ColumnReader reader = colReaders.get(descriptor); - for (int i = 0; i < rowsToWrite; i++) { - int rlvl = reader.getCurrentRepetitionLevel(); - int dlvl = reader.getCurrentDefinitionLevel(); - do { - if (dlvl < dMax) { - cWriter.writeNull(rlvl, dlvl); - } else if (columnType == Integer.TYPE) { - cWriter.write(reader.getInteger(), rlvl, dlvl); - } else if (columnType == Long.TYPE) { - cWriter.write(reader.getLong(), rlvl, dlvl); - } else if (columnType == Float.TYPE) { - cWriter.write(reader.getFloat(), rlvl, dlvl); - } else if (columnType == Double.TYPE) { - cWriter.write(reader.getDouble(), rlvl, dlvl); - } else if (columnType == Binary.class) { - cWriter.write(reader.getBinary(), rlvl, dlvl); - } else if (columnType == Boolean.TYPE) { - cWriter.write(reader.getBoolean(), rlvl, dlvl); - } else { - throw new UnsupportedOperationException( - String.format("Unsupported column java class: %s", columnType.toString())); - } - reader.consume(); - rlvl = reader.getCurrentRepetitionLevel(); - dlvl = reader.getCurrentDefinitionLevel(); - } while (rlvl > 0); - cStore.endRecord(); - } - } - - private MessageType newSchema(MessageType schema, ColumnDescriptor descriptor) { - String[] path = descriptor.getPath(); - Type type = schema.getType(path); - if (path.length == 1) { - return new MessageType(schema.getName(), type); - } - - for (Type field : schema.getFields()) { - if (!field.isPrimitive()) { - Type newType = extractField(field.asGroupType(), type); - if (newType != null) { - return new MessageType(schema.getName(), newType); - } - } - } - - // We should never hit this because 'type' is returned by schema.getType(). - throw new RuntimeException("No field is found"); - } - - private Type extractField(GroupType candidate, Type targetField) { - if (targetField.equals(candidate)) { - return targetField; - } - - // In case 'type' is a descendants of candidate - for (Type field : candidate.asGroupType().getFields()) { - if (field.isPrimitive()) { - if (field.equals(targetField)) { - return new GroupType(candidate.getRepetition(), candidate.getName(), targetField); - } - } else { - Type tempField = extractField(field.asGroupType(), targetField); - if (tempField != null) { - return new GroupType(candidate.getRepetition(), candidate.getName(), tempField); - } - } - } - - return null; - } - - private static final class DummyGroupConverter extends GroupConverter { - @Override - public void start() {} - - @Override - public void end() {} - - @Override - public Converter getConverter(int fieldIndex) { - return new DummyConverter(); - } - } - - private static final class DummyConverter extends PrimitiveConverter { - @Override - public GroupConverter asGroupConverter() { - return new DummyGroupConverter(); - } - } - - } - -} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index ed3dbc2aea..bd3686408e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -26,15 +26,10 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Preconditions; @@ -60,10 +55,7 @@ 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.IndexCache; -import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.*; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; @@ -76,6 +68,7 @@ import org.apache.parquet.io.InputFile; import org.apache.parquet.io.OutputFile; import org.apache.parquet.io.ParquetEncodingException; +import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.Converter; import org.apache.parquet.io.api.GroupConverter; import org.apache.parquet.io.api.PrimitiveConverter; @@ -106,6 +99,7 @@ public class ParquetRewriter implements Closeable { private int numBlocksRewritten = 0; // Reader and relevant states of the in-processing input file private final Queue inputFiles = new LinkedList<>(); + private final List columnWritersR = new ArrayList<>(); // Schema of input files (should be the same) and to write to the output file private MessageType schema = null; private final Map descriptorsMap; @@ -205,6 +199,117 @@ public ParquetRewriter( this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; } + // TODO converge with the main class constructor + public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException { + newCodecName = options.getNewCodecName(); + ParquetConfiguration conf = options.getParquetConfiguration(); + OutputFile outFile = options.getParquetOutputFile(); + inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); + List> inputFilesR = options.getParquetInputFilesR() + .stream() + .map(x -> getFileReaders(x, conf)) + .collect(Collectors.toList()); + ensureSameSchema(inputFiles); + inputFilesR.forEach(this::ensureSameSchema); + LOG.info("Start rewriting {} input file(s) {} to {}", inputFiles.size(), options.getParquetInputFiles(), outFile); // TODO add logging for all the files + + extraMetaData.put( + ORIGINAL_CREATED_BY_KEY, + Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .reduce((a, b) -> a + "\n" + b) + .orElse("") + ); + + // TODO check that schema on the left and on the right is not identical + MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); + List schemaR = inputFilesR + .stream() + .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) + .collect(Collectors.toList()); + + // TODO check that there is no overlap of fields on the right + Map fieldNamesL = new LinkedHashMap<>(); + schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); + Map fieldNamesR = new LinkedHashMap<>(); + schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); + List fields = Stream.concat( + fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can + fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left + ).collect(Collectors.toList()); + // Schema of input files (should be the same) and to write to the output file + MessageType schema = new MessageType(schemaL.getName(), fields); + + this.descriptorsMap = schemaL.getColumns().stream() + .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) + .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + this.indexCacheStrategy = options.getIndexCacheStrategy(); + + long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + inputFilesR.stream() + .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) + .forEach(rowCountR -> { + if (rowCountL != rowCountR) { + throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); + } + }); + + ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; + this.writer = new ParquetFileWriter( + outFile, + schema, + writerMode, + DEFAULT_BLOCK_SIZE, + MAX_PADDING_SIZE_DEFAULT, + DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, + DEFAULT_STATISTICS_TRUNCATE_LENGTH, + ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); + this.writer.start(); + + for (Queue inFiles: inputFilesR) { + this.columnWritersR.add(new RightColumnWriter(inFiles, this)); + } + } + + // TODO converge with the main class method + private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { + Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); + LinkedList inputFileReaders = new LinkedList<>(); + for (InputFile inputFile : inputFiles) { + try { + TransParquetFileReader reader = new TransParquetFileReader( + inputFile, ParquetReadOptions.builder(conf).build()); + inputFileReaders.add(reader); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to open input file: " + inputFile, e); + } + } + return inputFileReaders; + } + + // TODO converge with the main class method + private void ensureSameSchema(Queue inputFileReaders) { + MessageType schema = null; + for (TransParquetFileReader reader : inputFileReaders) { + MessageType newSchema = reader.getFooter().getFileMetaData().getSchema(); + if (schema == null) { + schema = newSchema; + } else { + // Now we enforce equality of schemas from input files for simplicity. + if (!schema.equals(newSchema)) { + String file = reader.getFile(); + LOG.error( + "Input files have different schemas, expect: {}, input: {}, current file: {}", + schema, + newSchema, + file); + throw new InvalidSchemaException( + "Input files have different schemas, current file: " + file); + } + } + } + } + // Open all input files to validate their schemas are compatible to merge private void openInputFiles(List inputFiles, ParquetConfiguration conf) { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); @@ -267,6 +372,12 @@ public void close() throws IOException { } public void processBlocks() throws IOException { + // TODO block processing implementations might need to be merged? + if (columnWritersR.isEmpty()) processBlocksDefault(); + else processBlocksWithJoin(); + } + + private void processBlocksDefault() throws IOException { while (reader != null) { IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); processBlocksFromReader(indexCache); @@ -813,7 +924,7 @@ private Type extractField(GroupType candidate, Type targetField) { } else { Type tempField = extractField(field.asGroupType(), targetField); if (tempField != null) { - return tempField; + return new GroupType(candidate.getRepetition(), candidate.getName(), tempField); } } } @@ -904,4 +1015,205 @@ public byte[] getDictPageAAD() { return this.dictPageAAD; } } + + private void processBlocksWithJoin() throws IOException { + // TODO add the test for empty files joins, it should merge schemas + int rowGroupIdx = 0; + while (!inputFiles.isEmpty()) { + TransParquetFileReader reader = inputFiles.poll(); + IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); + LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); + List blocks = reader.getFooter().getBlocks(); + for (BlockMetaData blockMetaData: blocks) { + writer.startBlock(blockMetaData.getRowCount()); + + // Writing the left side + indexCache.setBlockMetadata(blockMetaData); + List chunksL = blockMetaData.getColumns(); + for (ColumnChunkMetaData chunk : chunksL) { + if (chunk.isEncrypted()) { // TODO add that detail to docs + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); + } + ColumnDescriptor descriptorL = descriptorsMap.get(chunk.getPath()); + if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join + reader.setStreamPosition(chunk.getStartingPos()); + BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); + ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); + OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); + writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); + } + } + + // Writing the right side + for (RightColumnWriter writer: columnWritersR) { + writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); + } + + writer.endBlock(); + rowGroupIdx++; + } + } + } + + private static class RightColumnWriter { + private final Queue inputFiles; + private final ParquetRewriter parquetRewriter; + private final ParquetFileWriter writer; + private final MessageType schema; + private final Map descriptorsMap; + private final Map colReaders = new HashMap<>(); + private final Map cPageStores = new HashMap<>(); + private final Map cStores = new HashMap<>(); + private final Map cWriters = new HashMap<>(); + private int rowGroupIdxIn = 0; + private int rowGroupIdxOut = 0; + private int writtenFromBlock = 0; + + public RightColumnWriter(Queue inputFiles, ParquetRewriter parquetRewriter) throws IOException { + this.inputFiles = inputFiles; + this.parquetRewriter = parquetRewriter; + this.writer = parquetRewriter.writer; + this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); + this.descriptorsMap = + this.schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + initReaders(); + initWriters(); + } + + public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { + // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + if (rowGroupIdxIn != rowGroupIdx) { + this.rowGroupIdxIn = rowGroupIdx; + flushWriters(); + initWriters(); + } + while (rowsToWrite > 0) { + List blocks = inputFiles.peek().getFooter().getBlocks(); + BlockMetaData block = blocks.get(this.rowGroupIdxOut); + List chunks = block.getColumns(); + long leftInBlock = block.getRowCount() - writtenFromBlock; + long writeFromBlock = Math.min(rowsToWrite, leftInBlock); + for (ColumnChunkMetaData chunk : chunks) { + if (chunk.isEncrypted()) { // TODO check this during construction? + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs + } + ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); + copyValues(descriptor, writeFromBlock); + } + rowsToWrite -= writeFromBlock; + writtenFromBlock += writeFromBlock; + if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { + this.rowGroupIdxOut++; + if (this.rowGroupIdxOut == blocks.size()) { + inputFiles.poll(); + this.rowGroupIdxOut = 0; + } + writtenFromBlock = 0; + // this is called after all rows are processed + initReaders(); + } + } + flushWriters(); + } + + private void flushWriters() throws IOException { + cStores.values().forEach(cStore -> { + cStore.flush(); + cStore.close(); + }); + cWriters.values().forEach(ColumnWriter::close); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + if (cPageStores.containsKey(descriptor)) + cPageStores.get(descriptor).flushToFileWriter(writer); + } + cStores.clear(); + cWriters.clear(); + cPageStores.clear(); + } + + private void initWriters() { + if (!inputFiles.isEmpty()) { + List blocks = inputFiles.peek().getFooter().getBlocks(); + BlockMetaData block = blocks.get(this.rowGroupIdxOut); + ColumnChunkMetaData chunk = block.getColumns().get(0); // TODO use to current chunk idx? + ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = + ParquetProperties.builder().withWriterVersion(writerVersion).build(); + CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); + CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + MessageType columnSchema = parquetRewriter.newSchema(schema, descriptor); + ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( + compressor, + columnSchema, + props.getAllocator(), + props.getColumnIndexTruncateLength(), + props.getPageWriteChecksumEnabled(), + writer.getEncryptor(), + rowGroupIdxIn); + ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore); + ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); + cPageStores.put(descriptor, cPageStore); + cStores.put(descriptor, cwStore); + cWriters.put(descriptor, cWriter); + } + } + } + + private void initReaders() throws IOException { + if (!inputFiles.isEmpty()) { + TransParquetFileReader reader = inputFiles.peek(); + PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxOut); + String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); + ColumnReadStoreImpl crStore = + new ColumnReadStoreImpl(pageReadStore, new ParquetRewriter.DummyGroupConverter(), schema, createdBy); + for (ColumnDescriptor descriptor : descriptorsMap.values()) { + ColumnReader cReader = crStore.getColumnReader(descriptor); + colReaders.put(descriptor, cReader); + } + } + } + + private void copyValues( + ColumnDescriptor descriptor, + long rowsToWrite) { + ColumnWriteStore cStore = cStores.get(descriptor); + ColumnWriter cWriter = cWriters.get(descriptor); + int dMax = descriptor.getMaxDefinitionLevel(); + Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; + ColumnReader reader = colReaders.get(descriptor); + for (int i = 0; i < rowsToWrite; i++) { + int rlvl = reader.getCurrentRepetitionLevel(); + int dlvl = reader.getCurrentDefinitionLevel(); + do { + if (dlvl < dMax) { + cWriter.writeNull(rlvl, dlvl); + } else if (columnType == Integer.TYPE) { + cWriter.write(reader.getInteger(), rlvl, dlvl); + } else if (columnType == Long.TYPE) { + cWriter.write(reader.getLong(), rlvl, dlvl); + } else if (columnType == Float.TYPE) { + cWriter.write(reader.getFloat(), rlvl, dlvl); + } else if (columnType == Double.TYPE) { + cWriter.write(reader.getDouble(), rlvl, dlvl); + } else if (columnType == Binary.class) { + cWriter.write(reader.getBinary(), rlvl, dlvl); + } else if (columnType == Boolean.TYPE) { + cWriter.write(reader.getBoolean(), rlvl, dlvl); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported column java class: %s", columnType.toString())); + } + reader.consume(); + rlvl = reader.getCurrentRepetitionLevel(); + dlvl = reader.getCurrentDefinitionLevel(); + } while (rlvl > 0); + cStore.endRecord(); + } + } + + } + } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 127af68dd0..7996d35712 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -18,10 +18,7 @@ */ package org.apache.parquet.hadoop.rewrite; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -39,11 +36,20 @@ /** * A set of options to create a ParquetRewriter. + * + * TODO find a place where to put a proper description of functionality as it is not trivial: + * ParquetRewriter allows to stitch files with the same schema into a single file. + * Note that ParquetRewriter also can be used for effectively stitch/joining multiple parquet files with + * different schemas. + * You can provide the main input file group and multiple right side ones. That is possible when: + * 1) the number of rows in the main and extra input groups are the same, + * 2) the ordering of rows in the main and extra input groups is the same. */ public class RewriteOptions { private final ParquetConfiguration conf; private final List inputFiles; + private final List> inputFilesR; private final OutputFile outputFile; private final List pruneColumns; private final CompressionCodecName newCodecName; @@ -55,6 +61,7 @@ public class RewriteOptions { private RewriteOptions( ParquetConfiguration conf, List inputFiles, + List> inputFilesR, OutputFile outputFile, List pruneColumns, CompressionCodecName newCodecName, @@ -64,6 +71,7 @@ private RewriteOptions( IndexCache.CacheStrategy indexCacheStrategy) { this.conf = conf; this.inputFiles = inputFiles; + this.inputFilesR = inputFilesR; this.outputFile = outputFile; this.pruneColumns = pruneColumns; this.newCodecName = newCodecName; @@ -110,6 +118,27 @@ public List getInputFiles() { .collect(Collectors.toList()); } + /** TODO fix documentation after addition of inputFilesR + * Gets the right input {@link Path}s for the rewrite if they exist for all input files, + * otherwise throws a {@link RuntimeException}. + * + * @return a {@link List} of the associated right input {@link Path}s + */ + public List> getInputFilesR() { + return inputFilesR.stream() + .map(x -> x.stream() + .map(y -> { + if (y instanceof HadoopOutputFile) { + HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) y; + return new Path(hadoopOutputFile.getPath()); + } else { + throw new RuntimeException("The input files do not all have an associated Hadoop Path."); + } + }).collect(Collectors.toList()) + ) + .collect(Collectors.toList()); + } + /** * Gets the {@link InputFile}s for the rewrite. * @@ -119,6 +148,16 @@ public List getParquetInputFiles() { return inputFiles; } + + /** TODO fix documentation after addition of inputFilesR + * Gets the right {@link InputFile}s for the rewrite. + * + * @return a {@link List} of the associated right {@link InputFile}s + */ + public List> getParquetInputFilesR() { + return inputFilesR; + } + /** * Get the {@link Path} for the rewrite if it exists, otherwise throws a {@link RuntimeException}. * @@ -170,6 +209,7 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; + private final List> inputFilesR = new ArrayList<>(); private final OutputFile outputFile; private List pruneColumns; private CompressionCodecName newCodecName; @@ -325,6 +365,21 @@ public Builder addInputFile(Path path) { return this; } + /** TODO fix documentation after addition of inputFilesR + * Add an input file to read from. + * + * @param paths input file path to read from + * @return self + */ + public Builder addInputPathsR(List paths) { + this.inputFilesR.add( + paths.stream() + .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) + .collect(Collectors.toList()) + ); + return this; + } + /** * Add an input file to read from. * @@ -336,6 +391,17 @@ public Builder addInputFile(InputFile inputFile) { return this; } + /** TODO fix documentation after addition of inputFilesR + * Add an input file to read from. + * + * @param inputFiles input file to read from + * @return self + */ + public Builder addInputFilesR(List inputFiles) { + this.inputFilesR.add(inputFiles); + return this; + } + /** * Set the index(ColumnIndex, Offset and BloomFilter) cache strategy. *

@@ -358,6 +424,16 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); + Preconditions.checkArgument(inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), + "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); + Preconditions.checkArgument(inputFilesR.isEmpty() || pruneColumns == null, + "Right side Input files join functionality does not yet support column pruning"); + Preconditions.checkArgument(inputFilesR.isEmpty() || maskColumns == null, + "Right side Input files join functionality does not yet support column masking"); + Preconditions.checkArgument(inputFilesR.isEmpty() || encryptColumns == null, + "Right side Input files join functionality does not yet support column encryption"); + Preconditions.checkArgument(inputFilesR.isEmpty() || newCodecName == null, + "Right side Input files join functionality does not yet support codec changing"); if (pruneColumns != null) { if (maskColumns != null) { @@ -390,6 +466,7 @@ public RewriteOptions build() { return new RewriteOptions( conf, inputFiles, + inputFilesR, outputFile, pruneColumns, newCodecName, diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java index 64e5ddcad6..ffc5574c3b 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.Lists; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; @@ -41,6 +40,8 @@ import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.example.GroupReadSupport; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.rewrite.ParquetRewriter; +import org.apache.parquet.hadoop.rewrite.RewriteOptions; import org.apache.parquet.hadoop.util.EncryptionTestFile; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.hadoop.util.HadoopOutputFile; @@ -55,6 +56,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +// TODO move logic to ParquetRewriterTest @RunWith(Parameterized.class) public class ParquetJoinTest { @@ -68,7 +70,7 @@ public class ParquetJoinTest { private List inputFilesL = null; private List> inputFilesR = null; private String outputFile = null; - private ParquetJoiner joiner = null; + private ParquetRewriter rewriter = null; @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { @@ -106,12 +108,12 @@ public void testMergeTwoFilesOnly() throws Exception { List> inputPathsR = inputFilesR.stream() .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) .collect(Collectors.toList()); - JoinOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); - JoinOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); + RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); + RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); - joiner = new ParquetJoiner(options); - joiner.processBlocks(); - joiner.close(); + rewriter = new ParquetRewriter(options, true); + rewriter.processBlocks(); + rewriter.close(); // Verify the schema are not changed ParquetMetadata pmd = @@ -279,11 +281,12 @@ private void validateColumnData( } - private JoinOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { - JoinOptions.Builder builder; + private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { + RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); - builder = new JoinOptions.Builder(conf, inputPathsL, inputPathsR, outputPath); + builder = new RewriteOptions.Builder(conf, inputPathsL, outputPath); + inputPathsR.forEach(builder::addInputPathsR); } else { OutputFile outputPath = HadoopOutputFile.fromPath(new Path(outputFile), conf); List inputsL = inputPathsL.stream() @@ -293,7 +296,8 @@ private JoinOptions.Builder createBuilder(List inputPathsL, List x.stream().map(y -> (InputFile) HadoopInputFile.fromPathUnchecked(y, conf)).collect(Collectors.toList())) .collect(Collectors.toList()); - builder = new JoinOptions.Builder(parquetConf, inputsL, inputsR, outputPath); + builder = new RewriteOptions.Builder(parquetConf, inputsL, outputPath); + inputPathsR.forEach(builder::addInputPathsR); } return builder; } From d7f11d940a2aae973cb27ffb46c6f6f8496f866e Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 26 Feb 2024 21:47:05 -0800 Subject: [PATCH 07/57] move ParquetJoinTest logic to ParquetRewriterTest --- .../hadoop/rewrite/ParquetRewriter.java | 1 - .../parquet/hadoop/join/ParquetJoinTest.java | 305 ------------------ .../hadoop/rewrite/ParquetRewriterTest.java | 168 ++++++++-- 3 files changed, 148 insertions(+), 326 deletions(-) delete mode 100644 parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index bd3686408e..72421270df 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -29,7 +29,6 @@ import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Preconditions; diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java deleted file mode 100644 index ffc5574c3b..0000000000 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/join/ParquetJoinTest.java +++ /dev/null @@ -1,305 +0,0 @@ -/* - * 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.join; - -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; -import static org.apache.parquet.schema.Type.Repetition.*; -import static org.junit.Assert.*; - -import com.google.common.collect.Lists; -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.column.ParquetProperties; -import org.apache.parquet.conf.ParquetConfiguration; -import org.apache.parquet.conf.PlainParquetConfiguration; -import org.apache.parquet.crypto.FileDecryptionProperties; -import org.apache.parquet.example.data.Group; -import org.apache.parquet.example.data.simple.SimpleGroup; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.IndexCache; -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.ParquetMetadata; -import org.apache.parquet.hadoop.rewrite.ParquetRewriter; -import org.apache.parquet.hadoop.rewrite.RewriteOptions; -import org.apache.parquet.hadoop.util.EncryptionTestFile; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.hadoop.util.HadoopOutputFile; -import org.apache.parquet.hadoop.util.TestFileBuilder; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.OutputFile; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -// TODO move logic to ParquetRewriterTest -@RunWith(Parameterized.class) -public class ParquetJoinTest { - - private final int numRecord = 100_000; - private final Configuration conf = new Configuration(); - private final ParquetConfiguration parquetConf = new PlainParquetConfiguration(); - private final ParquetProperties.WriterVersion writerVersion; - private final IndexCache.CacheStrategy indexCacheStrategy; - private final boolean usingHadoop; - - private List inputFilesL = null; - private List> inputFilesR = null; - private String outputFile = null; - private ParquetRewriter rewriter = null; - - @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") - public static Object[][] parameters() { - return new Object[][] { - {"v1", "NONE", true}, - {"v1", "PREFETCH_BLOCK", true}, - {"v2", "NONE", true}, - {"v2", "PREFETCH_BLOCK", true}, - {"v1", "NONE", false}, - {"v1", "PREFETCH_BLOCK", false}, - {"v2", "NONE", false}, - {"v2", "PREFETCH_BLOCK", false} - }; - } - - @Before - public void setUp() { - outputFile = TestFileBuilder.createTempFile("test"); - } - - public ParquetJoinTest(String writerVersion, String indexCacheStrategy, boolean usingHadoop) { - this.writerVersion = ParquetProperties.WriterVersion.fromString(writerVersion); - this.indexCacheStrategy = IndexCache.CacheStrategy.valueOf(indexCacheStrategy); - this.usingHadoop = usingHadoop; - } - - @Test - public void testMergeTwoFilesOnly() throws Exception { - testMultiInputFileSetup(); - - // Only merge two files but do not change anything. - List inputPathsL = inputFilesL.stream() - .map(x -> new Path(x.getFileName())) - .collect(Collectors.toList()); - List> inputPathsR = inputFilesR.stream() - .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) - .collect(Collectors.toList()); - RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); - RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); - - rewriter = new ParquetRewriter(options, true); - rewriter.processBlocks(); - rewriter.close(); - - // Verify the schema are not changed - ParquetMetadata pmd = - ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); - MessageType schema = pmd.getFileMetaData().getSchema(); - MessageType expectSchema = createSchema(); - assertEquals(expectSchema, schema); - - // Verify the merged data are not changed - validateColumnData(null); - } - - private void testMultiInputFileSetup() throws IOException { - inputFilesL = Lists.newArrayList( - new TestFileBuilder(conf, createSchemaL()) - .withNumRecord(numRecord / 2) - .withRowGroupSize(5_000_000) - .withCodec("GZIP") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaL()) - .withNumRecord(numRecord - (numRecord / 2)) - .withRowGroupSize(6_000_000) - .withCodec("GZIP") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ); - inputFilesR = Lists.newArrayList( - Lists.newArrayList( - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR1()) - .withNumRecord(numRecord) - .withRowGroupSize(7_000_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ), - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(200_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(300_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord - 2 * (numRecord / 3)) - .withRowGroupSize(400_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ) - ) - ); - } - - private MessageType createSchema() { - return new MessageType( - "schema", - new PrimitiveType(OPTIONAL, INT64, "DocId"), - new PrimitiveType(REQUIRED, BINARY, "Name"), - new PrimitiveType(OPTIONAL, BINARY, "Gender"), - new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), - new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction"), - new GroupType( - OPTIONAL, - "Links", - new PrimitiveType(REPEATED, BINARY, "Backward"), - new PrimitiveType(REPEATED, BINARY, "Forward"))); - } - - private MessageType createSchemaL() { - return new MessageType( - "schema", - new PrimitiveType(OPTIONAL, INT64, "DocId"), - new PrimitiveType(REQUIRED, BINARY, "Name"), - new PrimitiveType(OPTIONAL, BINARY, "Gender"), - new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), - new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); - } - - private MessageType createSchemaR1() { - return new MessageType( - "schema", - new GroupType( - OPTIONAL, - "Links", - new PrimitiveType(REPEATED, BINARY, "Backward"), - new PrimitiveType(REPEATED, BINARY, "Forward"))); - } - - private MessageType createSchemaR2() { - return new MessageType( - "schema", - new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); - } - - private void validateColumnData( - FileDecryptionProperties fileDecryptionProperties) - throws IOException { - ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), new Path(outputFile)) - .withConf(conf) - .withDecryption(fileDecryptionProperties) - .build(); - - // Get total number of rows from input files - int totalRows = 0; - for (EncryptionTestFile inputFile : inputFilesL) { - totalRows += inputFile.getFileContent().length; - } - - int idxFileL = 0; - int idxFileR1 = 0; - int idxFileR2 = 0; - int idxRowL = 0; - int idxRowR1 = 0; - int idxRowR2 = 0; - for (int i = 0; i < totalRows; i++) { - Group group = reader.read(); - assertNotNull(group); - - if (idxRowL >= inputFilesL.get(idxFileL).getFileContent().length) { idxFileL++; idxRowL = 0; } - if (idxRowR1 >= inputFilesR.get(0).get(idxFileR1).getFileContent().length) { idxFileR1++; idxRowR1 = 0; } - if (idxRowR2 >= inputFilesR.get(1).get(idxFileR2).getFileContent().length) { idxFileR2++; idxRowR2 = 0; } - SimpleGroup expectGroupL = inputFilesL.get(idxFileL).getFileContent()[idxRowL++]; - SimpleGroup expectGroupR1 = inputFilesR.get(0).get(idxFileR1).getFileContent()[idxRowR1++]; - SimpleGroup expectGroupR2 = inputFilesR.get(1).get(idxFileR2).getFileContent()[idxRowR2++]; - - assertEquals(group.getLong("DocId", 0), expectGroupL.getLong("DocId", 0)); - assertArrayEquals( - group.getBinary("Name", 0).getBytes(), - expectGroupL.getBinary("Name", 0).getBytes()); - assertArrayEquals( - group.getBinary("Gender", 0).getBytes(), - expectGroupL.getBinary("Gender", 0).getBytes()); - assertEquals(expectGroupR2.getFloat("FloatFraction", 0), expectGroupR2.getFloat("FloatFraction", 0), 0); - assertEquals(group.getDouble("DoubleFraction", 0), expectGroupL.getDouble("DoubleFraction", 0), 0); - Group subGroup = group.getGroup("Links", 0); - assertArrayEquals( - subGroup.getBinary("Backward", 0).getBytes(), - expectGroupR1 - .getGroup("Links", 0) - .getBinary("Backward", 0) - .getBytes()); - assertArrayEquals( - subGroup.getBinary("Forward", 0).getBytes(), - expectGroupR1 - .getGroup("Links", 0) - .getBinary("Forward", 0) - .getBytes()); - } - - reader.close(); - } - - - private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { - RewriteOptions.Builder builder; - if (usingHadoop) { - Path outputPath = new Path(outputFile); - builder = new RewriteOptions.Builder(conf, inputPathsL, outputPath); - inputPathsR.forEach(builder::addInputPathsR); - } else { - OutputFile outputPath = HadoopOutputFile.fromPath(new Path(outputFile), conf); - List inputsL = inputPathsL.stream() - .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) - .collect(Collectors.toList()); - List> inputsR = inputPathsR - .stream() - .map(x -> x.stream().map(y -> (InputFile) HadoopInputFile.fromPathUnchecked(y, conf)).collect(Collectors.toList())) - .collect(Collectors.toList()); - builder = new RewriteOptions.Builder(parquetConf, inputsL, outputPath); - inputPathsR.forEach(builder::addInputPathsR); - } - return builder; - } - -} diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 2795a6ebac..b7a513384b 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -36,15 +36,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; +import java.util.function.BiFunction; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.HadoopReadOptions; @@ -108,6 +103,7 @@ public class ParquetRewriterTest { private final boolean usingHadoop; private List inputFiles = null; + private List> inputFilesR = null; private String outputFile = null; private ParquetRewriter rewriter = null; @@ -175,6 +171,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E @Before public void setUp() { outputFile = TestFileBuilder.createTempFile("test"); + inputFilesR = new ArrayList<>(); } @Test @@ -725,6 +722,90 @@ private void testMultipleInputFilesSetup() throws IOException { .build()); } + @Test + public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { + testThreeInputsDifferentRowGroupSize(); + + // Only merge two files but do not change anything. + List inputPathsL = inputFiles.stream() + .map(x -> new Path(x.getFileName())) + .collect(Collectors.toList()); + List> inputPathsR = inputFilesR.stream() + .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) + .collect(Collectors.toList()); + RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); + RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); + + rewriter = new ParquetRewriter(options, true); + rewriter.processBlocks(); + rewriter.close(); + + // Verify the schema are not changed + ParquetMetadata pmd = + ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); + MessageType schema = pmd.getFileMetaData().getSchema(); + MessageType expectSchema = createSchema(); + assertEquals(expectSchema, schema); + + // Verify the merged data are not changed + validateColumnData(Collections.emptySet(), Collections.emptySet(), null); + } + + private void testThreeInputsDifferentRowGroupSize() throws IOException { + inputFiles = Lists.newArrayList( + new TestFileBuilder(conf, createSchemaL()) + .withNumRecord(numRecord / 2) + .withRowGroupSize(5_000_000) + .withCodec("GZIP") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaL()) + .withNumRecord(numRecord - (numRecord / 2)) + .withRowGroupSize(6_000_000) + .withCodec("GZIP") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ); + inputFilesR = Lists.newArrayList( + Lists.newArrayList( + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR1()) + .withNumRecord(numRecord) + .withRowGroupSize(7_000_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ), + Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(200_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(300_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord - 2 * (numRecord / 3)) + .withRowGroupSize(400_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ) + ) + ); + } + private MessageType createSchema() { return new MessageType( "schema", @@ -740,6 +821,33 @@ private MessageType createSchema() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } + + private MessageType createSchemaL() { + return new MessageType( + "schema", + new PrimitiveType(OPTIONAL, INT64, "DocId"), + new PrimitiveType(REQUIRED, BINARY, "Name"), + new PrimitiveType(OPTIONAL, BINARY, "Gender"), + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), + new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); + } + + private MessageType createSchemaR1() { + return new MessageType( + "schema", + new GroupType( + OPTIONAL, + "Links", + new PrimitiveType(REPEATED, BINARY, "Backward"), + new PrimitiveType(REPEATED, BINARY, "Forward"))); + } + + private MessageType createSchemaR2() { + return new MessageType( + "schema", + new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); + } + private void validateColumnData( Set prunePaths, Set nullifiedPaths, FileDecryptionProperties fileDecryptionProperties) throws IOException { @@ -754,38 +862,51 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } + List> fileContents = Stream.concat( + Stream.of(inputFiles), + inputFilesR.stream() + ).map(x -> x.stream().flatMap(y -> Arrays.stream(y.getFileContent())).collect(Collectors.toList())) + .collect(Collectors.toList()); + BiFunction groups = (name, rowIdx) -> { + for (int i = fileContents.size() - 1; i >= 0; i--) { + SimpleGroup expGroup = fileContents.get(i).get(rowIdx); + GroupType fileSchema = expGroup.getType(); + if (fileSchema.containsField(name)) { + return expGroup; + } + } + throw new IllegalStateException("Group '"+name+"' at position "+rowIdx+" was not found in input files!"); + }; for (int i = 0; i < totalRows; i++) { Group group = reader.read(); assertNotNull(group); - SimpleGroup expectGroup = inputFiles.get(i / numRecord).getFileContent()[i % numRecord]; - if (!prunePaths.contains("DocId")) { if (nullifiedPaths.contains("DocId")) { assertThrows(RuntimeException.class, () -> group.getLong("DocId", 0)); } else { - assertEquals(group.getLong("DocId", 0), expectGroup.getLong("DocId", 0)); + assertEquals(group.getLong("DocId", 0), groups.apply("DocId", i).getLong("DocId", 0)); } } if (!prunePaths.contains("Name") && !nullifiedPaths.contains("Name")) { assertArrayEquals( group.getBinary("Name", 0).getBytes(), - expectGroup.getBinary("Name", 0).getBytes()); + groups.apply("Name", i).getBinary("Name", 0).getBytes()); } if (!prunePaths.contains("Gender") && !nullifiedPaths.contains("Gender")) { assertArrayEquals( group.getBinary("Gender", 0).getBytes(), - expectGroup.getBinary("Gender", 0).getBytes()); + groups.apply("Gender", i).getBinary("Gender", 0).getBytes()); } if (!prunePaths.contains("FloatFraction") && !nullifiedPaths.contains("FloatFraction")) { - assertEquals(group.getFloat("FloatFraction", 0), expectGroup.getFloat("FloatFraction", 0), 0); + assertEquals(group.getFloat("FloatFraction", 0), groups.apply("FloatFraction", i).getFloat("FloatFraction", 0), 0); } if (!prunePaths.contains("DoubleFraction") && !nullifiedPaths.contains("DoubleFraction")) { - assertEquals(group.getDouble("DoubleFraction", 0), expectGroup.getDouble("DoubleFraction", 0), 0); + assertEquals(group.getDouble("DoubleFraction", 0), groups.apply("DoubleFraction", i).getDouble("DoubleFraction", 0), 0); } Group subGroup = group.getGroup("Links", 0); @@ -793,7 +914,7 @@ private void validateColumnData( if (!prunePaths.contains("Links.Backward") && !nullifiedPaths.contains("Links.Backward")) { assertArrayEquals( subGroup.getBinary("Backward", 0).getBytes(), - expectGroup + groups.apply("Links", i) .getGroup("Links", 0) .getBinary("Backward", 0) .getBytes()); @@ -805,7 +926,7 @@ private void validateColumnData( } else { assertArrayEquals( subGroup.getBinary("Forward", 0).getBytes(), - expectGroup + groups.apply("Links", i) .getGroup("Links", 0) .getBinary("Forward", 0) .getBytes()); @@ -1042,16 +1163,22 @@ private Map> allBloomFilters( } private RewriteOptions.Builder createBuilder(List inputPaths) throws IOException { + return createBuilder(inputPaths, new ArrayList<>()); + } + + private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); - builder = new RewriteOptions.Builder(conf, inputPaths, outputPath); + builder = new RewriteOptions.Builder(conf, inputPathsL, outputPath); + inputPathsR.forEach(builder::addInputPathsR); } else { OutputFile outputPath = HadoopOutputFile.fromPath(new Path(outputFile), conf); - List inputs = inputPaths.stream() + List inputsL = inputPathsL.stream() .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) .collect(Collectors.toList()); - builder = new RewriteOptions.Builder(parquetConf, inputs, outputPath); + builder = new RewriteOptions.Builder(parquetConf, inputsL, outputPath); + inputPathsR.forEach(builder::addInputPathsR); } return builder; } @@ -1072,4 +1199,5 @@ private void validateSchema() throws IOException { assertEquals(subFields.get(0).getName(), "Backward"); assertEquals(subFields.get(1).getName(), "Forward"); } + } From e8e7ffe0b8b3217650fc1be1af487f484ce7dafb Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 29 Feb 2024 18:08:57 -0800 Subject: [PATCH 08/57] improve Parquet stitching test --- .../hadoop/rewrite/ParquetRewriter.java | 43 ++++++++----- .../hadoop/rewrite/ParquetRewriterTest.java | 63 +++++++++++++++---- 2 files changed, 77 insertions(+), 29 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 72421270df..d2689ac978 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -216,6 +216,8 @@ public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException ORIGINAL_CREATED_BY_KEY, Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .collect(Collectors.toSet()) + .stream() .reduce((a, b) -> a + "\n" + b) .orElse("") ); @@ -1082,13 +1084,13 @@ public RightColumnWriter(Queue inputFiles, ParquetRewrit public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); if (rowGroupIdxIn != rowGroupIdx) { - this.rowGroupIdxIn = rowGroupIdx; + rowGroupIdxIn = rowGroupIdx; flushWriters(); initWriters(); } while (rowsToWrite > 0) { List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(this.rowGroupIdxOut); + BlockMetaData block = blocks.get(rowGroupIdxOut); List chunks = block.getColumns(); long leftInBlock = block.getRowCount() - writtenFromBlock; long writeFromBlock = Math.min(rowsToWrite, leftInBlock); @@ -1102,10 +1104,10 @@ public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { rowsToWrite -= writeFromBlock; writtenFromBlock += writeFromBlock; if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { - this.rowGroupIdxOut++; - if (this.rowGroupIdxOut == blocks.size()) { + rowGroupIdxOut++; + if (rowGroupIdxOut == blocks.size()) { inputFiles.poll(); - this.rowGroupIdxOut = 0; + rowGroupIdxOut = 0; } writtenFromBlock = 0; // this is called after all rows are processed @@ -1133,16 +1135,23 @@ private void flushWriters() throws IOException { private void initWriters() { if (!inputFiles.isEmpty()) { List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(this.rowGroupIdxOut); - ColumnChunkMetaData chunk = block.getColumns().get(0); // TODO use to current chunk idx? - ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = - ParquetProperties.builder().withWriterVersion(writerVersion).build(); - CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { + descriptorsMap.forEach((columnPath, descriptor) -> { + ColumnChunkMetaData chunk = blocks.get(rowGroupIdxOut).getColumns().stream() + .filter(x -> x.getPath() == columnPath) + .findFirst() + .orElseThrow(() -> new IllegalStateException("Could not find column [" + columnPath.toDotString() + "].")); + int bloomFilterLength = chunk.getBloomFilterLength(); + ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = + ParquetProperties.builder() + .withWriterVersion(writerVersion) + .withBloomFilterEnabled(bloomFilterLength > 0) + .build(); + CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); + CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + MessageType columnSchema = parquetRewriter.newSchema(schema, descriptor); ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( compressor, @@ -1152,12 +1161,12 @@ private void initWriters() { props.getPageWriteChecksumEnabled(), writer.getEncryptor(), rowGroupIdxIn); - ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore); + ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore, cPageStore); ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); cPageStores.put(descriptor, cPageStore); cStores.put(descriptor, cwStore); cWriters.put(descriptor, cWriter); - } + }); } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index b7a513384b..30f12590b9 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -42,6 +42,7 @@ import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Version; @@ -726,7 +727,6 @@ private void testMultipleInputFilesSetup() throws IOException { public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { testThreeInputsDifferentRowGroupSize(); - // Only merge two files but do not change anything. List inputPathsL = inputFiles.stream() .map(x -> new Path(x.getFileName())) .collect(Collectors.toList()); @@ -739,16 +739,46 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { rewriter = new ParquetRewriter(options, true); rewriter.processBlocks(); rewriter.close(); - - // Verify the schema are not changed - ParquetMetadata pmd = - ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); - MessageType schema = pmd.getFileMetaData().getSchema(); + MessageType actualSchema = + ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) + .getFileMetaData() + .getSchema(); MessageType expectSchema = createSchema(); - assertEquals(expectSchema, schema); - // Verify the merged data are not changed - validateColumnData(Collections.emptySet(), Collections.emptySet(), null); + Map> inputBloomFilters = allInputBloomFilters(null); + Map> outputBloomFilters = allOutputBloomFilters(null); + Set schemaR1Columns = createSchemaR1().getColumns().stream().map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + Set schemaR2Columns = createSchemaR2().getColumns().stream().map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + Set r1BloomFilters = outputBloomFilters.keySet().stream().filter(schemaR1Columns::contains).collect(Collectors.toSet()); + Set r2withBloomFilters = outputBloomFilters.keySet().stream().filter(schemaR2Columns::contains).collect(Collectors.toSet()); + Set rBloomFilters = Stream.concat( + r1BloomFilters.stream(), + r2withBloomFilters.stream() + ).collect(Collectors.toSet()); + + // TODO potentially too many checks, might need to be split into multiple tests + validateColumnData(Collections.emptySet(), Collections.emptySet(), null); // Verify data + assertEquals(expectSchema, actualSchema); // Verify schema + validateCreatedBy(); // Verify original.created.by + assertEquals(inputBloomFilters.keySet(), rBloomFilters); // Verify bloom filters + verifyCodec( // Verify codec + outputFile, + new HashSet() { + { + add(CompressionCodecName.GZIP); + add(CompressionCodecName.UNCOMPRESSED); + add(CompressionCodecName.ZSTD); + } + }, + null); + validatePageIndex(new HashMap() { // Verify page index + { // verifying only left side input columns + put(0, 0); + put(1, 1); + put(2, 2); + put(3, 4); + } + }); } private void testThreeInputsDifferentRowGroupSize() throws IOException { @@ -774,9 +804,10 @@ private void testThreeInputsDifferentRowGroupSize() throws IOException { new TestFileBuilder(conf, createSchemaR1()) .withNumRecord(numRecord) .withRowGroupSize(7_000_000) - .withCodec("UNCOMPRESSED") + .withCodec("ZSTD") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) + .withBloomFilterEnabled(new String[]{"Links.Forward"}) .build() ), Lists.newArrayList( @@ -1076,7 +1107,11 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); - for (EncryptionTestFile inputFile : inputFiles) { + List inFiles = Stream.concat( + inputFiles.stream(), + inputFilesR.stream().flatMap(Collection::stream) + ).collect(Collectors.toList()); + for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); createdBySet.add(pmd.getFileMetaData().getCreatedBy()); assertNull(pmd.getFileMetaData().getKeyValueMetaData().get(ParquetRewriter.ORIGINAL_CREATED_BY_KEY)); @@ -1119,7 +1154,11 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters(FileDecryptionProperties fileDecryptionProperties) throws Exception { Map> inputBloomFilters = new HashMap<>(); - for (EncryptionTestFile inputFile : inputFiles) { + List files = Stream.concat( + Stream.of(inputFiles), + inputFilesR.stream() + ).flatMap(Collection::stream).collect(Collectors.toList()); + for (EncryptionTestFile inputFile : files) { Map> bloomFilters = allBloomFilters(inputFile.getFileName(), fileDecryptionProperties); for (Map.Entry> entry : bloomFilters.entrySet()) { From 3ee946cc0446f6ba49d08e70b18792efb459a8a7 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 5 Mar 2024 23:20:32 -0800 Subject: [PATCH 09/57] remove custom ParquetRewriter constructor --- .../hadoop/rewrite/ParquetRewriter.java | 208 +++++++++--------- .../hadoop/rewrite/ParquetRewriterTest.java | 3 +- 2 files changed, 101 insertions(+), 110 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index d2689ac978..bb9706f594 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -108,19 +108,52 @@ public class ParquetRewriter implements Closeable { private ParquetMetadata meta = null; // created_by information of current reader being processed private String originalCreatedBy = ""; - // Unique created_by information from all input files - private final Set allOriginalCreatedBys = new HashSet<>(); // The index cache strategy private final IndexCache.CacheStrategy indexCacheStrategy; public ParquetRewriter(RewriteOptions options) throws IOException { ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); - openInputFiles(options.getParquetInputFiles(), conf); + inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); + List> inputFilesR = options.getParquetInputFilesR() + .stream() + .map(x -> getFileReaders(x, conf)) + .collect(Collectors.toList()); + ensureSameSchema(inputFiles); + inputFilesR.forEach(this::ensureSameSchema); LOG.info("Start rewriting {} input file(s) {} to {}", inputFiles.size(), options.getParquetInputFiles(), out); + extraMetaData.put( + ORIGINAL_CREATED_BY_KEY, + Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .collect(Collectors.toSet()) + .stream() + .reduce((a, b) -> a + "\n" + b) + .orElse("") + ); + Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); + // Init reader of the first input file - initNextReader(); +// initNextReader(); + + // TODO check that schema on the left and on the right is not identical + MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); + List schemaR = inputFilesR + .stream() + .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) + .collect(Collectors.toList()); + // TODO check that there is no overlap of fields on the right + Map fieldNamesL = new LinkedHashMap<>(); + schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); + Map fieldNamesR = new LinkedHashMap<>(); + schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); + List fields = Stream.concat( + fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can + fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left + ).collect(Collectors.toList()); + schema = new MessageType(schemaL.getName(), fields); newCodecName = options.getNewCodecName(); List pruneColumns = options.getPruneColumns(); @@ -130,7 +163,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { getPaths(schema, paths, null); for (String col : pruneColumns) { if (!paths.contains(col)) { - LOG.warn("Input column name {} doesn't show up in the schema of file {}", col, reader.getFile()); + LOG.warn("Input column name {} doesn't show up in the schema", col); } } @@ -138,8 +171,23 @@ public ParquetRewriter(RewriteOptions options) throws IOException { schema = pruneColumnsInSchema(schema, prunePaths); } - this.descriptorsMap = - schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + if (inputFilesR.isEmpty()) { // TODO: find a more suitable solution + this.descriptorsMap = + schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + } else { + this.descriptorsMap = schemaL.getColumns().stream() + .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) + .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + } + + long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + inputFilesR.stream() + .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) + .forEach(rowCountR -> { + if (rowCountL != rowCountR) { + throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); + } + }); if (options.getMaskColumns() != null) { this.maskColumns = new HashMap<>(); @@ -167,6 +215,10 @@ public ParquetRewriter(RewriteOptions options) throws IOException { ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED, options.getFileEncryptionProperties()); writer.start(); + + for (Queue inFiles: inputFilesR) { + this.columnWritersR.add(new RightColumnWriter(inFiles, this)); + } } // Ctor for legacy CompressionConverter and ColumnMasker @@ -221,6 +273,8 @@ public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException .reduce((a, b) -> a + "\n" + b) .orElse("") ); + Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); // TODO check that schema on the left and on the right is not identical MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); @@ -244,7 +298,6 @@ public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException this.descriptorsMap = schemaL.getColumns().stream() .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - this.indexCacheStrategy = options.getIndexCacheStrategy(); long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); inputFilesR.stream() @@ -255,8 +308,10 @@ public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException } }); + this.indexCacheStrategy = options.getIndexCacheStrategy(); + ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; - this.writer = new ParquetFileWriter( + writer = new ParquetFileWriter( outFile, schema, writerMode, @@ -265,16 +320,15 @@ public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, DEFAULT_STATISTICS_TRUNCATE_LENGTH, ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); - this.writer.start(); + writer.start(); for (Queue inFiles: inputFilesR) { this.columnWritersR.add(new RightColumnWriter(inFiles, this)); } } - // TODO converge with the main class method private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { - Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); + // Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); // TODO: remove, this is already checked in RewriteOptions LinkedList inputFileReaders = new LinkedList<>(); for (InputFile inputFile : inputFiles) { try { @@ -288,7 +342,6 @@ private Queue getFileReaders(List inputFiles, return inputFileReaders; } - // TODO converge with the main class method private void ensureSameSchema(Queue inputFileReaders) { MessageType schema = null; for (TransParquetFileReader reader : inputFileReaders) { @@ -311,79 +364,22 @@ private void ensureSameSchema(Queue inputFileReaders) { } } - // Open all input files to validate their schemas are compatible to merge - private void openInputFiles(List inputFiles, ParquetConfiguration conf) { - Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); - - for (InputFile inputFile : inputFiles) { - try { - TransParquetFileReader reader = new TransParquetFileReader( - inputFile, ParquetReadOptions.builder(conf).build()); - MessageType inputFileSchema = - reader.getFooter().getFileMetaData().getSchema(); - if (this.schema == null) { - this.schema = inputFileSchema; - } else { - // Now we enforce equality of schemas from input files for simplicity. - if (!this.schema.equals(inputFileSchema)) { - LOG.error( - "Input files have different schemas, expect: {}, input: {}, current file: {}", - this.schema, - inputFileSchema, - inputFile); - throw new InvalidSchemaException( - "Input files have different schemas, current file: " + inputFile); - } - } - this.allOriginalCreatedBys.add( - reader.getFooter().getFileMetaData().getCreatedBy()); - this.inputFiles.add(reader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to open input file: " + inputFile, e); - } - } - - extraMetaData.put(ORIGINAL_CREATED_BY_KEY, String.join("\n", allOriginalCreatedBys)); - } - - // Routines to get reader of next input file and set up relevant states - private void initNextReader() { - if (reader != null) { - LOG.info("Finish rewriting input file: {}", reader.getFile()); - } - - if (inputFiles.isEmpty()) { - reader = null; - meta = null; - originalCreatedBy = null; - return; - } - - reader = inputFiles.poll(); - meta = reader.getFooter(); - originalCreatedBy = meta.getFileMetaData().getCreatedBy(); - extraMetaData.putAll(meta.getFileMetaData().getKeyValueMetaData()); - - LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); - } - @Override public void close() throws IOException { writer.end(extraMetaData); } public void processBlocks() throws IOException { - // TODO block processing implementations might need to be merged? - if (columnWritersR.isEmpty()) processBlocksDefault(); - else processBlocksWithJoin(); - } - - private void processBlocksDefault() throws IOException { - while (reader != null) { + while (!inputFiles.isEmpty()) { + reader = inputFiles.poll(); + meta = reader.getFooter(); + originalCreatedBy = meta.getFileMetaData().getCreatedBy(); + LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); - processBlocksFromReader(indexCache); + if (columnWritersR.isEmpty()) processBlocksFromReader(indexCache); + else processBlocksWithJoin(indexCache); indexCache.clean(); - initNextReader(); + LOG.info("Finish rewriting input file: {}", reader.getFile()); } } @@ -1017,42 +1013,38 @@ public byte[] getDictPageAAD() { } } - private void processBlocksWithJoin() throws IOException { + private void processBlocksWithJoin(IndexCache indexCache) throws IOException { // TODO add the test for empty files joins, it should merge schemas + LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); int rowGroupIdx = 0; - while (!inputFiles.isEmpty()) { - TransParquetFileReader reader = inputFiles.poll(); - IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); - LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); - List blocks = reader.getFooter().getBlocks(); - for (BlockMetaData blockMetaData: blocks) { - writer.startBlock(blockMetaData.getRowCount()); - - // Writing the left side - indexCache.setBlockMetadata(blockMetaData); - List chunksL = blockMetaData.getColumns(); - for (ColumnChunkMetaData chunk : chunksL) { - if (chunk.isEncrypted()) { // TODO add that detail to docs - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); - } - ColumnDescriptor descriptorL = descriptorsMap.get(chunk.getPath()); - if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join - reader.setStreamPosition(chunk.getStartingPos()); - BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); - ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); - OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); - } - } + List blocks = reader.getFooter().getBlocks(); + for (BlockMetaData blockMetaData: blocks) { + writer.startBlock(blockMetaData.getRowCount()); - // Writing the right side - for (RightColumnWriter writer: columnWritersR) { - writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); + // Writing the left side + indexCache.setBlockMetadata(blockMetaData); + List chunksL = blockMetaData.getColumns(); + for (ColumnChunkMetaData chunk : chunksL) { + if (chunk.isEncrypted()) { // TODO add that detail to docs + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); + } + ColumnDescriptor descriptorL = descriptorsMap.get(chunk.getPath()); + if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join + reader.setStreamPosition(chunk.getStartingPos()); + BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); + ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); + OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); + writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } + } - writer.endBlock(); - rowGroupIdx++; + // Writing the right side + for (RightColumnWriter writer: columnWritersR) { + writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); } + + writer.endBlock(); + rowGroupIdx++; } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 30f12590b9..47fb488479 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -42,7 +42,6 @@ import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Version; @@ -736,7 +735,7 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); - rewriter = new ParquetRewriter(options, true); + rewriter = new ParquetRewriter(options); rewriter.processBlocks(); rewriter.close(); MessageType actualSchema = From fd409c4bd3136a45f5feca14c8bc93d8f418616d Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 5 Mar 2024 23:23:07 -0800 Subject: [PATCH 10/57] remove custom ParquetRewriter constructor --- .../hadoop/rewrite/ParquetRewriter.java | 80 ------------------- 1 file changed, 80 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index bb9706f594..1f540fedda 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -135,9 +135,6 @@ public ParquetRewriter(RewriteOptions options) throws IOException { Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); - // Init reader of the first input file -// initNextReader(); - // TODO check that schema on the left and on the right is not identical MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); List schemaR = inputFilesR @@ -250,83 +247,6 @@ public ParquetRewriter( this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; } - // TODO converge with the main class constructor - public ParquetRewriter(RewriteOptions options, boolean dummy) throws IOException { - newCodecName = options.getNewCodecName(); - ParquetConfiguration conf = options.getParquetConfiguration(); - OutputFile outFile = options.getParquetOutputFile(); - inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - List> inputFilesR = options.getParquetInputFilesR() - .stream() - .map(x -> getFileReaders(x, conf)) - .collect(Collectors.toList()); - ensureSameSchema(inputFiles); - inputFilesR.forEach(this::ensureSameSchema); - LOG.info("Start rewriting {} input file(s) {} to {}", inputFiles.size(), options.getParquetInputFiles(), outFile); // TODO add logging for all the files - - extraMetaData.put( - ORIGINAL_CREATED_BY_KEY, - Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) - .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) - .collect(Collectors.toSet()) - .stream() - .reduce((a, b) -> a + "\n" + b) - .orElse("") - ); - Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) - .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); - - // TODO check that schema on the left and on the right is not identical - MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - List schemaR = inputFilesR - .stream() - .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) - .collect(Collectors.toList()); - - // TODO check that there is no overlap of fields on the right - Map fieldNamesL = new LinkedHashMap<>(); - schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); - Map fieldNamesR = new LinkedHashMap<>(); - schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); - List fields = Stream.concat( - fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can - fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left - ).collect(Collectors.toList()); - // Schema of input files (should be the same) and to write to the output file - MessageType schema = new MessageType(schemaL.getName(), fields); - - this.descriptorsMap = schemaL.getColumns().stream() - .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) - .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - - long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - inputFilesR.stream() - .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) - .forEach(rowCountR -> { - if (rowCountL != rowCountR) { - throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); - } - }); - - this.indexCacheStrategy = options.getIndexCacheStrategy(); - - ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; - writer = new ParquetFileWriter( - outFile, - schema, - writerMode, - DEFAULT_BLOCK_SIZE, - MAX_PADDING_SIZE_DEFAULT, - DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH, - DEFAULT_STATISTICS_TRUNCATE_LENGTH, - ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); - writer.start(); - - for (Queue inFiles: inputFilesR) { - this.columnWritersR.add(new RightColumnWriter(inFiles, this)); - } - } - private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { // Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); // TODO: remove, this is already checked in RewriteOptions LinkedList inputFileReaders = new LinkedList<>(); From 5a982190f1643eefd839f3c08a6207684c25e7b5 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 11 Mar 2024 21:27:51 -0700 Subject: [PATCH 11/57] refactor ParquetRewriter --- .../org/apache/parquet/hadoop/rewrite/ParquetRewriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 1f540fedda..d0f037fba5 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -297,7 +297,7 @@ public void processBlocks() throws IOException { LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); if (columnWritersR.isEmpty()) processBlocksFromReader(indexCache); - else processBlocksWithJoin(indexCache); + else processBlocksFromReaderWithStitching(indexCache); indexCache.clean(); LOG.info("Finish rewriting input file: {}", reader.getFile()); } @@ -933,7 +933,7 @@ public byte[] getDictPageAAD() { } } - private void processBlocksWithJoin(IndexCache indexCache) throws IOException { + private void processBlocksFromReaderWithStitching(IndexCache indexCache) throws IOException { // TODO add the test for empty files joins, it should merge schemas LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); int rowGroupIdx = 0; From 7b2fd1abc52ac0d3bc2936f011d8c8fa30f32b9b Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 13 Mar 2024 21:42:24 -0700 Subject: [PATCH 12/57] apply spotless and address PR comments --- .../hadoop/rewrite/ParquetRewriter.java | 102 +++++----- .../hadoop/rewrite/RewriteOptions.java | 69 ++----- .../hadoop/rewrite/ParquetRewriterTest.java | 178 +++++++++--------- 3 files changed, 170 insertions(+), 179 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index d0f037fba5..605f4eb780 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -26,7 +26,16 @@ import java.io.Closeable; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; @@ -115,8 +124,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - List> inputFilesR = options.getParquetInputFilesR() - .stream() + List> inputFilesR = options.getParquetInputFilesR().stream() .map(x -> getFileReaders(x, conf)) .collect(Collectors.toList()); ensureSameSchema(inputFiles); @@ -130,15 +138,13 @@ public ParquetRewriter(RewriteOptions options) throws IOException { .collect(Collectors.toSet()) .stream() .reduce((a, b) -> a + "\n" + b) - .orElse("") - ); + .orElse("")); Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); // TODO check that schema on the left and on the right is not identical MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - List schemaR = inputFilesR - .stream() + List schemaR = inputFilesR.stream() .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) .collect(Collectors.toList()); // TODO check that there is no overlap of fields on the right @@ -147,9 +153,14 @@ public ParquetRewriter(RewriteOptions options) throws IOException { Map fieldNamesR = new LinkedHashMap<>(); schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); List fields = Stream.concat( - fieldNamesL.values().stream().map(x -> fieldNamesR.getOrDefault(x.getName(), x)), // take a field on the right if we can - fieldNamesR.values().stream().filter(x -> !fieldNamesL.containsKey(x.getName())) // takes fields on the right if it was not present on the left - ).collect(Collectors.toList()); + fieldNamesL.values().stream() + .map(x -> fieldNamesR.getOrDefault( + x.getName(), x)), // take a field on the right if we can + fieldNamesR.values().stream() + .filter(x -> !fieldNamesL.containsKey( + x.getName())) // takes fields on the right if it was not present on the left + ) + .collect(Collectors.toList()); schema = new MessageType(schemaL.getName(), fields); newCodecName = options.getNewCodecName(); @@ -173,16 +184,19 @@ public ParquetRewriter(RewriteOptions options) throws IOException { schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); } else { this.descriptorsMap = schemaL.getColumns().stream() - .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) + .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); } - long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); + long rowCountL = + inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); inputFilesR.stream() - .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) + .map(x -> + x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) .forEach(rowCountR -> { if (rowCountL != rowCountR) { - throw new IllegalArgumentException("The number of records on the left and on the right don't match!"); + throw new IllegalArgumentException("The number of records on the left(" + rowCountL + + ") and on the right(" + rowCountR + ") don't match!"); } }); @@ -213,7 +227,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { options.getFileEncryptionProperties()); writer.start(); - for (Queue inFiles: inputFilesR) { + for (Queue inFiles : inputFilesR) { this.columnWritersR.add(new RightColumnWriter(inFiles, this)); } } @@ -248,7 +262,8 @@ public ParquetRewriter( } private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { - // Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); // TODO: remove, this is already checked in RewriteOptions + // Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); // TODO: remove, + // this is already checked in RewriteOptions LinkedList inputFileReaders = new LinkedList<>(); for (InputFile inputFile : inputFiles) { try { @@ -277,8 +292,7 @@ private void ensureSameSchema(Queue inputFileReaders) { schema, newSchema, file); - throw new InvalidSchemaException( - "Input files have different schemas, current file: " + file); + throw new InvalidSchemaException("Input files have different schemas, current file: " + file); } } } @@ -938,7 +952,7 @@ private void processBlocksFromReaderWithStitching(IndexCache indexCache) throws LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); int rowGroupIdx = 0; List blocks = reader.getFooter().getBlocks(); - for (BlockMetaData blockMetaData: blocks) { + for (BlockMetaData blockMetaData : blocks) { writer.startBlock(blockMetaData.getRowCount()); // Writing the left side @@ -954,12 +968,13 @@ private void processBlocksFromReaderWithStitching(IndexCache indexCache) throws BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk(descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); + writer.appendColumnChunk( + descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } } // Writing the right side - for (RightColumnWriter writer: columnWritersR) { + for (RightColumnWriter writer : columnWritersR) { writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); } @@ -982,19 +997,21 @@ private static class RightColumnWriter { private int rowGroupIdxOut = 0; private int writtenFromBlock = 0; - public RightColumnWriter(Queue inputFiles, ParquetRewriter parquetRewriter) throws IOException { + public RightColumnWriter(Queue inputFiles, ParquetRewriter parquetRewriter) + throws IOException { this.inputFiles = inputFiles; this.parquetRewriter = parquetRewriter; this.writer = parquetRewriter.writer; this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - this.descriptorsMap = - this.schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + this.descriptorsMap = this.schema.getColumns().stream() + .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); initReaders(); initWriters(); } public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { - // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), inputFilesR.size()); + // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), + // inputFilesR.size()); if (rowGroupIdxIn != rowGroupIdx) { rowGroupIdxIn = rowGroupIdx; flushWriters(); @@ -1008,7 +1025,8 @@ public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { long writeFromBlock = Math.min(rowsToWrite, leftInBlock); for (ColumnChunkMetaData chunk : chunks) { if (chunk.isEncrypted()) { // TODO check this during construction? - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); // TODO add that detail to docs + throw new IOException("Column " + chunk.getPath().toDotString() + + " is encrypted"); // TODO add that detail to docs } ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); copyValues(descriptor, writeFromBlock); @@ -1051,18 +1069,20 @@ private void initWriters() { ColumnChunkMetaData chunk = blocks.get(rowGroupIdxOut).getColumns().stream() .filter(x -> x.getPath() == columnPath) .findFirst() - .orElseThrow(() -> new IllegalStateException("Could not find column [" + columnPath.toDotString() + "].")); + .orElseThrow(() -> new IllegalStateException( + "Could not find column [" + columnPath.toDotString() + "].")); int bloomFilterLength = chunk.getBloomFilterLength(); - ParquetProperties.WriterVersion writerVersion = chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = - ParquetProperties.builder() - .withWriterVersion(writerVersion) - .withBloomFilterEnabled(bloomFilterLength > 0) - .build(); + ParquetProperties.WriterVersion writerVersion = + chunk.getEncodingStats().usesV2Pages() + ? ParquetProperties.WriterVersion.PARQUET_2_0 + : ParquetProperties.WriterVersion.PARQUET_1_0; + ParquetProperties props = ParquetProperties.builder() + .withWriterVersion(writerVersion) + .withBloomFilterEnabled(bloomFilterLength > 0) + .build(); CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = codecFactory.getCompressor(chunk.getCodec()); + CompressionCodecFactory.BytesInputCompressor compressor = + codecFactory.getCompressor(chunk.getCodec()); MessageType columnSchema = parquetRewriter.newSchema(schema, descriptor); ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( @@ -1087,8 +1107,8 @@ private void initReaders() throws IOException { TransParquetFileReader reader = inputFiles.peek(); PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxOut); String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); - ColumnReadStoreImpl crStore = - new ColumnReadStoreImpl(pageReadStore, new ParquetRewriter.DummyGroupConverter(), schema, createdBy); + ColumnReadStoreImpl crStore = new ColumnReadStoreImpl( + pageReadStore, new ParquetRewriter.DummyGroupConverter(), schema, createdBy); for (ColumnDescriptor descriptor : descriptorsMap.values()) { ColumnReader cReader = crStore.getColumnReader(descriptor); colReaders.put(descriptor, cReader); @@ -1096,9 +1116,7 @@ private void initReaders() throws IOException { } } - private void copyValues( - ColumnDescriptor descriptor, - long rowsToWrite) { + private void copyValues(ColumnDescriptor descriptor, long rowsToWrite) { ColumnWriteStore cStore = cStores.get(descriptor); ColumnWriter cWriter = cWriters.get(descriptor); int dMax = descriptor.getMaxDefinitionLevel(); @@ -1133,7 +1151,5 @@ private void copyValues( cStore.endRecord(); } } - } - } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 7996d35712..d0baf522cb 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -18,7 +18,10 @@ */ package org.apache.parquet.hadoop.rewrite; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -99,46 +102,6 @@ public ParquetConfiguration getParquetConfiguration() { return conf; } - /** - * Gets the input {@link Path}s for the rewrite if they exist for all input files, - * otherwise throws a {@link RuntimeException}. - * - * @return a {@link List} of the associated input {@link Path}s - */ - public List getInputFiles() { - return inputFiles.stream() - .map(f -> { - if (f instanceof HadoopOutputFile) { - HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) f; - return new Path(hadoopOutputFile.getPath()); - } else { - throw new RuntimeException("The input files do not all have an associated Hadoop Path."); - } - }) - .collect(Collectors.toList()); - } - - /** TODO fix documentation after addition of inputFilesR - * Gets the right input {@link Path}s for the rewrite if they exist for all input files, - * otherwise throws a {@link RuntimeException}. - * - * @return a {@link List} of the associated right input {@link Path}s - */ - public List> getInputFilesR() { - return inputFilesR.stream() - .map(x -> x.stream() - .map(y -> { - if (y instanceof HadoopOutputFile) { - HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) y; - return new Path(hadoopOutputFile.getPath()); - } else { - throw new RuntimeException("The input files do not all have an associated Hadoop Path."); - } - }).collect(Collectors.toList()) - ) - .collect(Collectors.toList()); - } - /** * Gets the {@link InputFile}s for the rewrite. * @@ -148,7 +111,6 @@ public List getParquetInputFiles() { return inputFiles; } - /** TODO fix documentation after addition of inputFilesR * Gets the right {@link InputFile}s for the rewrite. * @@ -372,11 +334,9 @@ public Builder addInputFile(Path path) { * @return self */ public Builder addInputPathsR(List paths) { - this.inputFilesR.add( - paths.stream() - .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) - .collect(Collectors.toList()) - ); + this.inputFilesR.add(paths.stream() + .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) + .collect(Collectors.toList())); return this; } @@ -424,15 +384,20 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); - Preconditions.checkArgument(inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), + Preconditions.checkArgument( + inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); - Preconditions.checkArgument(inputFilesR.isEmpty() || pruneColumns == null, + Preconditions.checkArgument( + inputFilesR.isEmpty() || pruneColumns == null, "Right side Input files join functionality does not yet support column pruning"); - Preconditions.checkArgument(inputFilesR.isEmpty() || maskColumns == null, + Preconditions.checkArgument( + inputFilesR.isEmpty() || maskColumns == null, "Right side Input files join functionality does not yet support column masking"); - Preconditions.checkArgument(inputFilesR.isEmpty() || encryptColumns == null, + Preconditions.checkArgument( + inputFilesR.isEmpty() || encryptColumns == null, "Right side Input files join functionality does not yet support column encryption"); - Preconditions.checkArgument(inputFilesR.isEmpty() || newCodecName == null, + Preconditions.checkArgument( + inputFilesR.isEmpty() || newCodecName == null, "Right side Input files join functionality does not yet support codec changing"); if (pruneColumns != null) { diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 47fb488479..77fe7ff616 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -36,7 +36,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -726,9 +734,8 @@ private void testMultipleInputFilesSetup() throws IOException { public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { testThreeInputsDifferentRowGroupSize(); - List inputPathsL = inputFiles.stream() - .map(x -> new Path(x.getFileName())) - .collect(Collectors.toList()); + List inputPathsL = + inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); List> inputPathsR = inputFilesR.stream() .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) .collect(Collectors.toList()); @@ -738,22 +745,28 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { rewriter = new ParquetRewriter(options); rewriter.processBlocks(); rewriter.close(); - MessageType actualSchema = - ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) - .getFileMetaData() - .getSchema(); + MessageType actualSchema = ParquetFileReader.readFooter( + conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) + .getFileMetaData() + .getSchema(); MessageType expectSchema = createSchema(); Map> inputBloomFilters = allInputBloomFilters(null); Map> outputBloomFilters = allOutputBloomFilters(null); - Set schemaR1Columns = createSchemaR1().getColumns().stream().map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); - Set schemaR2Columns = createSchemaR2().getColumns().stream().map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); - Set r1BloomFilters = outputBloomFilters.keySet().stream().filter(schemaR1Columns::contains).collect(Collectors.toSet()); - Set r2withBloomFilters = outputBloomFilters.keySet().stream().filter(schemaR2Columns::contains).collect(Collectors.toSet()); - Set rBloomFilters = Stream.concat( - r1BloomFilters.stream(), - r2withBloomFilters.stream() - ).collect(Collectors.toSet()); + Set schemaR1Columns = createSchemaR1().getColumns().stream() + .map(x -> ColumnPath.get(x.getPath())) + .collect(Collectors.toSet()); + Set schemaR2Columns = createSchemaR2().getColumns().stream() + .map(x -> ColumnPath.get(x.getPath())) + .collect(Collectors.toSet()); + Set r1BloomFilters = outputBloomFilters.keySet().stream() + .filter(schemaR1Columns::contains) + .collect(Collectors.toSet()); + Set r2withBloomFilters = outputBloomFilters.keySet().stream() + .filter(schemaR2Columns::contains) + .collect(Collectors.toSet()); + Set rBloomFilters = Stream.concat(r1BloomFilters.stream(), r2withBloomFilters.stream()) + .collect(Collectors.toSet()); // TODO potentially too many checks, might need to be split into multiple tests validateColumnData(Collections.emptySet(), Collections.emptySet(), null); // Verify data @@ -770,14 +783,15 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { } }, null); - validatePageIndex(new HashMap() { // Verify page index - { // verifying only left side input columns - put(0, 0); - put(1, 1); - put(2, 2); - put(3, 4); - } - }); + validatePageIndex( + new HashMap() { // Verify page index + { // verifying only left side input columns + put(0, 0); + put(1, 1); + put(2, 2); + put(3, 4); + } + }); } private void testThreeInputsDifferentRowGroupSize() throws IOException { @@ -795,45 +809,38 @@ private void testThreeInputsDifferentRowGroupSize() throws IOException { .withCodec("GZIP") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) - .build() - ); - inputFilesR = Lists.newArrayList( + .build()); + inputFilesR = Lists.newArrayList(Lists.newArrayList( + Lists.newArrayList(new TestFileBuilder(conf, createSchemaR1()) + .withNumRecord(numRecord) + .withRowGroupSize(7_000_000) + .withCodec("ZSTD") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .withBloomFilterEnabled(new String[] {"Links.Forward"}) + .build()), Lists.newArrayList( - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR1()) - .withNumRecord(numRecord) - .withRowGroupSize(7_000_000) - .withCodec("ZSTD") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .withBloomFilterEnabled(new String[]{"Links.Forward"}) - .build() - ), - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(200_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(300_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord - 2 * (numRecord / 3)) - .withRowGroupSize(400_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ) - ) - ); + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(200_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord / 3) + .withRowGroupSize(300_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR2()) + .withNumRecord(numRecord - 2 * (numRecord / 3)) + .withRowGroupSize(400_000) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()))); } private MessageType createSchema() { @@ -851,7 +858,6 @@ private MessageType createSchema() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } - private MessageType createSchemaL() { return new MessageType( "schema", @@ -873,9 +879,7 @@ private MessageType createSchemaR1() { } private MessageType createSchemaR2() { - return new MessageType( - "schema", - new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); + return new MessageType("schema", new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); } private void validateColumnData( @@ -892,10 +896,10 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } - List> fileContents = Stream.concat( - Stream.of(inputFiles), - inputFilesR.stream() - ).map(x -> x.stream().flatMap(y -> Arrays.stream(y.getFileContent())).collect(Collectors.toList())) + List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesR.stream()) + .map(x -> x.stream() + .flatMap(y -> Arrays.stream(y.getFileContent())) + .collect(Collectors.toList())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { for (int i = fileContents.size() - 1; i >= 0; i--) { @@ -905,7 +909,8 @@ private void validateColumnData( return expGroup; } } - throw new IllegalStateException("Group '"+name+"' at position "+rowIdx+" was not found in input files!"); + throw new IllegalStateException( + "Group '" + name + "' at position " + rowIdx + " was not found in input files!"); }; for (int i = 0; i < totalRows; i++) { Group group = reader.read(); @@ -915,7 +920,8 @@ private void validateColumnData( if (nullifiedPaths.contains("DocId")) { assertThrows(RuntimeException.class, () -> group.getLong("DocId", 0)); } else { - assertEquals(group.getLong("DocId", 0), groups.apply("DocId", i).getLong("DocId", 0)); + assertEquals( + group.getLong("DocId", 0), groups.apply("DocId", i).getLong("DocId", 0)); } } @@ -932,11 +938,17 @@ private void validateColumnData( } if (!prunePaths.contains("FloatFraction") && !nullifiedPaths.contains("FloatFraction")) { - assertEquals(group.getFloat("FloatFraction", 0), groups.apply("FloatFraction", i).getFloat("FloatFraction", 0), 0); + assertEquals( + group.getFloat("FloatFraction", 0), + groups.apply("FloatFraction", i).getFloat("FloatFraction", 0), + 0); } if (!prunePaths.contains("DoubleFraction") && !nullifiedPaths.contains("DoubleFraction")) { - assertEquals(group.getDouble("DoubleFraction", 0), groups.apply("DoubleFraction", i).getDouble("DoubleFraction", 0), 0); + assertEquals( + group.getDouble("DoubleFraction", 0), + groups.apply("DoubleFraction", i).getDouble("DoubleFraction", 0), + 0); } Group subGroup = group.getGroup("Links", 0); @@ -1106,10 +1118,9 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); - List inFiles = Stream.concat( - inputFiles.stream(), - inputFilesR.stream().flatMap(Collection::stream) - ).collect(Collectors.toList()); + List inFiles = Stream.concat( + inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + .collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); createdBySet.add(pmd.getFileMetaData().getCreatedBy()); @@ -1153,10 +1164,9 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters(FileDecryptionProperties fileDecryptionProperties) throws Exception { Map> inputBloomFilters = new HashMap<>(); - List files = Stream.concat( - Stream.of(inputFiles), - inputFilesR.stream() - ).flatMap(Collection::stream).collect(Collectors.toList()); + List files = Stream.concat(Stream.of(inputFiles), inputFilesR.stream()) + .flatMap(Collection::stream) + .collect(Collectors.toList()); for (EncryptionTestFile inputFile : files) { Map> bloomFilters = allBloomFilters(inputFile.getFileName(), fileDecryptionProperties); @@ -1204,7 +1214,8 @@ private RewriteOptions.Builder createBuilder(List inputPaths) throws IOExc return createBuilder(inputPaths, new ArrayList<>()); } - private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) throws IOException { + private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) + throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); @@ -1237,5 +1248,4 @@ private void validateSchema() throws IOException { assertEquals(subFields.get(0).getName(), "Backward"); assertEquals(subFields.get(1).getName(), "Forward"); } - } From 8da82910b5dd60aeb2d2ce973556c049b82339a1 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Fri, 15 Mar 2024 16:20:05 -0700 Subject: [PATCH 13/57] move extra column writing into processBlocksFromReader --- .../hadoop/rewrite/ParquetRewriter.java | 46 ++++--------------- 1 file changed, 8 insertions(+), 38 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 605f4eb780..f19ad8980b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -310,14 +310,14 @@ public void processBlocks() throws IOException { originalCreatedBy = meta.getFileMetaData().getCreatedBy(); LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); - if (columnWritersR.isEmpty()) processBlocksFromReader(indexCache); - else processBlocksFromReaderWithStitching(indexCache); + processBlocksFromReader(indexCache); indexCache.clean(); LOG.info("Finish rewriting input file: {}", reader.getFile()); } } private void processBlocksFromReader(IndexCache indexCache) throws IOException { + int rowGroupIdx = 0; for (int blockId = 0; blockId < meta.getBlocks().size(); blockId++) { BlockMetaData blockMetaData = meta.getBlocks().get(blockId); writer.startBlock(blockMetaData.getRowCount()); @@ -389,8 +389,14 @@ private void processBlocksFromReader(IndexCache indexCache) throws IOException { columnId++; } + // Writing extra columns + for (RightColumnWriter writer : columnWritersR) { + writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); + } + writer.endBlock(); numBlocksRewritten++; + rowGroupIdx++; } } @@ -947,42 +953,6 @@ public byte[] getDictPageAAD() { } } - private void processBlocksFromReaderWithStitching(IndexCache indexCache) throws IOException { - // TODO add the test for empty files joins, it should merge schemas - LOG.info("Rewriting input fileLeft: {}, remaining filesLeft: {}", reader.getFile(), inputFiles.size()); - int rowGroupIdx = 0; - List blocks = reader.getFooter().getBlocks(); - for (BlockMetaData blockMetaData : blocks) { - writer.startBlock(blockMetaData.getRowCount()); - - // Writing the left side - indexCache.setBlockMetadata(blockMetaData); - List chunksL = blockMetaData.getColumns(); - for (ColumnChunkMetaData chunk : chunksL) { - if (chunk.isEncrypted()) { // TODO add that detail to docs - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); - } - ColumnDescriptor descriptorL = descriptorsMap.get(chunk.getPath()); - if (descriptorL != null) { // descriptorL might be NULL if a column is from the right side of a join - reader.setStreamPosition(chunk.getStartingPos()); - BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); - ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); - OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk( - descriptorL, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); - } - } - - // Writing the right side - for (RightColumnWriter writer : columnWritersR) { - writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); - } - - writer.endBlock(); - rowGroupIdx++; - } - } - private static class RightColumnWriter { private final Queue inputFiles; private final ParquetRewriter parquetRewriter; From 68e41ba094d348653e273c3bc25e120a7d5591c5 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Fri, 15 Mar 2024 18:32:33 -0700 Subject: [PATCH 14/57] add getInputFiles back --- .../hadoop/rewrite/RewriteOptions.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index d0baf522cb..4012e90f50 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -102,6 +102,25 @@ public ParquetConfiguration getParquetConfiguration() { return conf; } + /** + * Gets the input {@link Path}s for the rewrite if they exist for all input files, + * otherwise throws a {@link RuntimeException}. + * + * @return a {@link List} of the associated input {@link Path}s + */ + public List getInputFiles() { + return inputFiles.stream() + .map(f -> { + if (f instanceof HadoopOutputFile) { + HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) f; + return new Path(hadoopOutputFile.getPath()); + } else { + throw new RuntimeException("The input files do not all have an associated Hadoop Path."); + } + }) + .collect(Collectors.toList()); + } + /** * Gets the {@link InputFile}s for the rewrite. * From 6d2c2229b1579f7149233b8013f00e1fb1a31218 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sat, 16 Mar 2024 12:05:19 -0700 Subject: [PATCH 15/57] fix extra ParquetRewriter constructor so tests can pass --- .../org/apache/parquet/hadoop/rewrite/ParquetRewriter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index f19ad8980b..a9843799a1 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -242,9 +242,7 @@ public ParquetRewriter( CompressionCodecName codecName, List maskColumns, MaskMode maskMode) { - this.reader = reader; this.writer = writer; - this.meta = meta; this.schema = schema; this.descriptorsMap = schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); @@ -258,6 +256,7 @@ public ParquetRewriter( this.maskColumns.put(ColumnPath.fromDotString(col), maskMode); } } + this.inputFiles.add(reader); this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; } From 883e93586589f2027e9187d2b4eeda8c4e8e2b89 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 19 Mar 2024 22:25:35 -0700 Subject: [PATCH 16/57] remove not needed TODOs --- .../hadoop/rewrite/ParquetRewriter.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index a9843799a1..950280b086 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -142,16 +142,20 @@ public ParquetRewriter(RewriteOptions options) throws IOException { Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); - // TODO check that schema on the left and on the right is not identical MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); List schemaR = inputFilesR.stream() .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) .collect(Collectors.toList()); - // TODO check that there is no overlap of fields on the right Map fieldNamesL = new LinkedHashMap<>(); schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); Map fieldNamesR = new LinkedHashMap<>(); - schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> fieldNamesR.put(x.getName(), x)); + schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> { + if (fieldNamesR.containsKey(x.getName())) { + throw new IllegalArgumentException( + "Found a duplicated field `" + x.getName() + "` in the right side file groups!"); + } + fieldNamesR.put(x.getName(), x); + }); List fields = Stream.concat( fieldNamesL.values().stream() .map(x -> fieldNamesR.getOrDefault( @@ -179,10 +183,10 @@ public ParquetRewriter(RewriteOptions options) throws IOException { schema = pruneColumnsInSchema(schema, prunePaths); } - if (inputFilesR.isEmpty()) { // TODO: find a more suitable solution + if (inputFilesR.isEmpty()) { this.descriptorsMap = schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - } else { + } else { // TODO: describe in documentation that only top level column can be overwritten this.descriptorsMap = schemaL.getColumns().stream() .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); @@ -261,8 +265,6 @@ public ParquetRewriter( } private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { - // Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "No input files"); // TODO: remove, - // this is already checked in RewriteOptions LinkedList inputFileReaders = new LinkedList<>(); for (InputFile inputFile : inputFiles) { try { @@ -979,8 +981,6 @@ public RightColumnWriter(Queue inputFiles, ParquetRewrit } public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { - // LOG.info("Rewriting input fileRight: {}, remaining fileRight: {}", readerR.getFile(), - // inputFilesR.size()); if (rowGroupIdxIn != rowGroupIdx) { rowGroupIdxIn = rowGroupIdx; flushWriters(); @@ -993,9 +993,8 @@ public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { long leftInBlock = block.getRowCount() - writtenFromBlock; long writeFromBlock = Math.min(rowsToWrite, leftInBlock); for (ColumnChunkMetaData chunk : chunks) { - if (chunk.isEncrypted()) { // TODO check this during construction? - throw new IOException("Column " + chunk.getPath().toDotString() - + " is encrypted"); // TODO add that detail to docs + if (chunk.isEncrypted()) { + throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); } ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); copyValues(descriptor, writeFromBlock); From 8ef36b59a99f51c92607b00de44905d0a32222e3 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 24 Mar 2024 14:03:00 -0700 Subject: [PATCH 17/57] address PR comments --- .../hadoop/rewrite/ParquetRewriter.java | 38 +++++++++++-------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 950280b086..2b1f9a71d3 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -63,7 +63,11 @@ import org.apache.parquet.format.DictionaryPageHeader; import org.apache.parquet.format.PageHeader; import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.*; +import org.apache.parquet.hadoop.CodecFactory; +import org.apache.parquet.hadoop.ColumnChunkPageWriteStore; +import org.apache.parquet.hadoop.IndexCache; +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; @@ -318,7 +322,6 @@ public void processBlocks() throws IOException { } private void processBlocksFromReader(IndexCache indexCache) throws IOException { - int rowGroupIdx = 0; for (int blockId = 0; blockId < meta.getBlocks().size(); blockId++) { BlockMetaData blockMetaData = meta.getBlocks().get(blockId); writer.startBlock(blockMetaData.getRowCount()); @@ -391,13 +394,12 @@ private void processBlocksFromReader(IndexCache indexCache) throws IOException { } // Writing extra columns - for (RightColumnWriter writer : columnWritersR) { - writer.writeRows(rowGroupIdx, blockMetaData.getRowCount()); + for (RightColumnWriter columnWriter : columnWritersR) { + columnWriter.writeRows(numBlocksRewritten, blockMetaData.getRowCount()); } writer.endBlock(); numBlocksRewritten++; - rowGroupIdx++; } } @@ -964,8 +966,8 @@ private static class RightColumnWriter { private final Map cPageStores = new HashMap<>(); private final Map cStores = new HashMap<>(); private final Map cWriters = new HashMap<>(); - private int rowGroupIdxIn = 0; - private int rowGroupIdxOut = 0; + private int rowGroupIdxL = 0; // index of the rowGroup of the current file on the left + private int rowGroupIdxR = 0; // index of the rowGroup of the current file on the right private int writtenFromBlock = 0; public RightColumnWriter(Queue inputFiles, ParquetRewriter parquetRewriter) @@ -981,14 +983,18 @@ public RightColumnWriter(Queue inputFiles, ParquetRewrit } public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { - if (rowGroupIdxIn != rowGroupIdx) { - rowGroupIdxIn = rowGroupIdx; + if (rowGroupIdxL > rowGroupIdx) { + throw new IOException("A row group index decrease is determined in RightColumnWriter! Current index: " + + rowGroupIdxL + ", new index: " + rowGroupIdx); + } + if (rowGroupIdxL != rowGroupIdx) { + rowGroupIdxL = rowGroupIdx; flushWriters(); initWriters(); } while (rowsToWrite > 0) { List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(rowGroupIdxOut); + BlockMetaData block = blocks.get(rowGroupIdxR); List chunks = block.getColumns(); long leftInBlock = block.getRowCount() - writtenFromBlock; long writeFromBlock = Math.min(rowsToWrite, leftInBlock); @@ -1002,10 +1008,10 @@ public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { rowsToWrite -= writeFromBlock; writtenFromBlock += writeFromBlock; if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { - rowGroupIdxOut++; - if (rowGroupIdxOut == blocks.size()) { + rowGroupIdxR++; + if (rowGroupIdxR == blocks.size()) { inputFiles.poll(); - rowGroupIdxOut = 0; + rowGroupIdxR = 0; } writtenFromBlock = 0; // this is called after all rows are processed @@ -1034,7 +1040,7 @@ private void initWriters() { if (!inputFiles.isEmpty()) { List blocks = inputFiles.peek().getFooter().getBlocks(); descriptorsMap.forEach((columnPath, descriptor) -> { - ColumnChunkMetaData chunk = blocks.get(rowGroupIdxOut).getColumns().stream() + ColumnChunkMetaData chunk = blocks.get(rowGroupIdxR).getColumns().stream() .filter(x -> x.getPath() == columnPath) .findFirst() .orElseThrow(() -> new IllegalStateException( @@ -1060,7 +1066,7 @@ private void initWriters() { props.getColumnIndexTruncateLength(), props.getPageWriteChecksumEnabled(), writer.getEncryptor(), - rowGroupIdxIn); + rowGroupIdxL); ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore, cPageStore); ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); cPageStores.put(descriptor, cPageStore); @@ -1073,7 +1079,7 @@ private void initWriters() { private void initReaders() throws IOException { if (!inputFiles.isEmpty()) { TransParquetFileReader reader = inputFiles.peek(); - PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxOut); + PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxR); String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); ColumnReadStoreImpl crStore = new ColumnReadStoreImpl( pageReadStore, new ParquetRewriter.DummyGroupConverter(), schema, createdBy); From 0bbf72f27184aed9f86d8251b175fcebe2d3bb72 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 10 Apr 2024 18:54:41 -0700 Subject: [PATCH 18/57] rename inputFilesR to inputFilesToJoin --- .../hadoop/rewrite/ParquetRewriter.java | 16 ++++----- .../hadoop/rewrite/RewriteOptions.java | 36 +++++++++---------- .../hadoop/rewrite/ParquetRewriterTest.java | 14 ++++---- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 2b1f9a71d3..710f93c934 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -128,26 +128,26 @@ public ParquetRewriter(RewriteOptions options) throws IOException { ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - List> inputFilesR = options.getParquetInputFilesR().stream() + List> inputFilesToJoin = options.getParquetInputFilesToJoin().stream() .map(x -> getFileReaders(x, conf)) .collect(Collectors.toList()); ensureSameSchema(inputFiles); - inputFilesR.forEach(this::ensureSameSchema); + inputFilesToJoin.forEach(this::ensureSameSchema); LOG.info("Start rewriting {} input file(s) {} to {}", inputFiles.size(), options.getParquetInputFiles(), out); extraMetaData.put( ORIGINAL_CREATED_BY_KEY, - Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) .collect(Collectors.toSet()) .stream() .reduce((a, b) -> a + "\n" + b) .orElse("")); - Stream.concat(inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - List schemaR = inputFilesR.stream() + List schemaR = inputFilesToJoin.stream() .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) .collect(Collectors.toList()); Map fieldNamesL = new LinkedHashMap<>(); @@ -187,7 +187,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { schema = pruneColumnsInSchema(schema, prunePaths); } - if (inputFilesR.isEmpty()) { + if (inputFilesToJoin.isEmpty()) { this.descriptorsMap = schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); } else { // TODO: describe in documentation that only top level column can be overwritten @@ -198,7 +198,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { long rowCountL = inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - inputFilesR.stream() + inputFilesToJoin.stream() .map(x -> x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) .forEach(rowCountR -> { @@ -235,7 +235,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { options.getFileEncryptionProperties()); writer.start(); - for (Queue inFiles : inputFilesR) { + for (Queue inFiles : inputFilesToJoin) { this.columnWritersR.add(new RightColumnWriter(inFiles, this)); } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 4012e90f50..8b7a8345fb 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -52,7 +52,7 @@ public class RewriteOptions { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesR; + private final List> inputFilesToJoin; private final OutputFile outputFile; private final List pruneColumns; private final CompressionCodecName newCodecName; @@ -64,7 +64,7 @@ public class RewriteOptions { private RewriteOptions( ParquetConfiguration conf, List inputFiles, - List> inputFilesR, + List> inputFilesToJoin, OutputFile outputFile, List pruneColumns, CompressionCodecName newCodecName, @@ -74,7 +74,7 @@ private RewriteOptions( IndexCache.CacheStrategy indexCacheStrategy) { this.conf = conf; this.inputFiles = inputFiles; - this.inputFilesR = inputFilesR; + this.inputFilesToJoin = inputFilesToJoin; this.outputFile = outputFile; this.pruneColumns = pruneColumns; this.newCodecName = newCodecName; @@ -130,13 +130,13 @@ public List getParquetInputFiles() { return inputFiles; } - /** TODO fix documentation after addition of inputFilesR + /** TODO fix documentation after addition of InputFilesToJoin * Gets the right {@link InputFile}s for the rewrite. * * @return a {@link List} of the associated right {@link InputFile}s */ - public List> getParquetInputFilesR() { - return inputFilesR; + public List> getParquetInputFilesToJoin() { + return inputFilesToJoin; } /** @@ -190,7 +190,7 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesR = new ArrayList<>(); + private final List> inputFilesToJoin = new ArrayList<>(); private final OutputFile outputFile; private List pruneColumns; private CompressionCodecName newCodecName; @@ -346,14 +346,14 @@ public Builder addInputFile(Path path) { return this; } - /** TODO fix documentation after addition of inputFilesR + /** TODO fix documentation after addition of InputFilesToJoin * Add an input file to read from. * * @param paths input file path to read from * @return self */ public Builder addInputPathsR(List paths) { - this.inputFilesR.add(paths.stream() + this.inputFilesToJoin.add(paths.stream() .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) .collect(Collectors.toList())); return this; @@ -370,14 +370,14 @@ public Builder addInputFile(InputFile inputFile) { return this; } - /** TODO fix documentation after addition of inputFilesR + /** TODO fix documentation after addition of InputFilesToJoin * Add an input file to read from. * * @param inputFiles input file to read from * @return self */ - public Builder addInputFilesR(List inputFiles) { - this.inputFilesR.add(inputFiles); + public Builder addInputFilesToJoin(List inputFiles) { + this.inputFilesToJoin.add(inputFiles); return this; } @@ -404,19 +404,19 @@ public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); Preconditions.checkArgument( - inputFilesR.stream().allMatch(x -> x != null && !x.isEmpty()), + inputFilesToJoin.stream().allMatch(x -> x != null && !x.isEmpty()), "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); Preconditions.checkArgument( - inputFilesR.isEmpty() || pruneColumns == null, + inputFilesToJoin.isEmpty() || pruneColumns == null, "Right side Input files join functionality does not yet support column pruning"); Preconditions.checkArgument( - inputFilesR.isEmpty() || maskColumns == null, + inputFilesToJoin.isEmpty() || maskColumns == null, "Right side Input files join functionality does not yet support column masking"); Preconditions.checkArgument( - inputFilesR.isEmpty() || encryptColumns == null, + inputFilesToJoin.isEmpty() || encryptColumns == null, "Right side Input files join functionality does not yet support column encryption"); Preconditions.checkArgument( - inputFilesR.isEmpty() || newCodecName == null, + inputFilesToJoin.isEmpty() || newCodecName == null, "Right side Input files join functionality does not yet support codec changing"); if (pruneColumns != null) { @@ -450,7 +450,7 @@ public RewriteOptions build() { return new RewriteOptions( conf, inputFiles, - inputFilesR, + inputFilesToJoin, outputFile, pruneColumns, newCodecName, diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 77fe7ff616..946509f98b 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -111,7 +111,7 @@ public class ParquetRewriterTest { private final boolean usingHadoop; private List inputFiles = null; - private List> inputFilesR = null; + private List> inputFilesToJoin = null; private String outputFile = null; private ParquetRewriter rewriter = null; @@ -179,7 +179,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E @Before public void setUp() { outputFile = TestFileBuilder.createTempFile("test"); - inputFilesR = new ArrayList<>(); + inputFilesToJoin = new ArrayList<>(); } @Test @@ -736,7 +736,7 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); - List> inputPathsR = inputFilesR.stream() + List> inputPathsR = inputFilesToJoin.stream() .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) .collect(Collectors.toList()); RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); @@ -810,7 +810,7 @@ private void testThreeInputsDifferentRowGroupSize() throws IOException { .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - inputFilesR = Lists.newArrayList(Lists.newArrayList( + inputFilesToJoin = Lists.newArrayList(Lists.newArrayList( Lists.newArrayList(new TestFileBuilder(conf, createSchemaR1()) .withNumRecord(numRecord) .withRowGroupSize(7_000_000) @@ -896,7 +896,7 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } - List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesR.stream()) + List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesToJoin.stream()) .map(x -> x.stream() .flatMap(y -> Arrays.stream(y.getFileContent())) .collect(Collectors.toList())) @@ -1119,7 +1119,7 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); List inFiles = Stream.concat( - inputFiles.stream(), inputFilesR.stream().flatMap(Collection::stream)) + inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); @@ -1164,7 +1164,7 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters(FileDecryptionProperties fileDecryptionProperties) throws Exception { Map> inputBloomFilters = new HashMap<>(); - List files = Stream.concat(Stream.of(inputFiles), inputFilesR.stream()) + List files = Stream.concat(Stream.of(inputFiles), inputFilesToJoin.stream()) .flatMap(Collection::stream) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : files) { From ca53bffd8f4111209ea1acdc8646383c3d691989 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 10 Apr 2024 20:01:57 -0700 Subject: [PATCH 19/57] rename inputFilesR to inputFilesToJoinColumns --- .../hadoop/rewrite/ParquetRewriter.java | 2 +- .../hadoop/rewrite/RewriteOptions.java | 36 +++++++++---------- .../hadoop/rewrite/ParquetRewriterTest.java | 14 ++++---- 3 files changed, 26 insertions(+), 26 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 710f93c934..3a0a8ca8da 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -128,7 +128,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - List> inputFilesToJoin = options.getParquetInputFilesToJoin().stream() + List> inputFilesToJoin = options.getParquetInputFilesToJoinColumns().stream() .map(x -> getFileReaders(x, conf)) .collect(Collectors.toList()); ensureSameSchema(inputFiles); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 8b7a8345fb..7ba527e3f0 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -52,7 +52,7 @@ public class RewriteOptions { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesToJoin; + private final List> inputFilesToJoinColumns; private final OutputFile outputFile; private final List pruneColumns; private final CompressionCodecName newCodecName; @@ -64,7 +64,7 @@ public class RewriteOptions { private RewriteOptions( ParquetConfiguration conf, List inputFiles, - List> inputFilesToJoin, + List> inputFilesToJoinColumns, OutputFile outputFile, List pruneColumns, CompressionCodecName newCodecName, @@ -74,7 +74,7 @@ private RewriteOptions( IndexCache.CacheStrategy indexCacheStrategy) { this.conf = conf; this.inputFiles = inputFiles; - this.inputFilesToJoin = inputFilesToJoin; + this.inputFilesToJoinColumns = inputFilesToJoinColumns; this.outputFile = outputFile; this.pruneColumns = pruneColumns; this.newCodecName = newCodecName; @@ -130,13 +130,13 @@ public List getParquetInputFiles() { return inputFiles; } - /** TODO fix documentation after addition of InputFilesToJoin + /** TODO fix documentation after addition of InputFilesToJoinColumns * Gets the right {@link InputFile}s for the rewrite. * * @return a {@link List} of the associated right {@link InputFile}s */ - public List> getParquetInputFilesToJoin() { - return inputFilesToJoin; + public List> getParquetInputFilesToJoinColumns() { + return inputFilesToJoinColumns; } /** @@ -190,7 +190,7 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesToJoin = new ArrayList<>(); + private final List> inputFilesToJoinColumns = new ArrayList<>(); private final OutputFile outputFile; private List pruneColumns; private CompressionCodecName newCodecName; @@ -346,14 +346,14 @@ public Builder addInputFile(Path path) { return this; } - /** TODO fix documentation after addition of InputFilesToJoin + /** TODO fix documentation after addition of InputFilesToJoinColumns * Add an input file to read from. * * @param paths input file path to read from * @return self */ public Builder addInputPathsR(List paths) { - this.inputFilesToJoin.add(paths.stream() + this.inputFilesToJoinColumns.add(paths.stream() .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) .collect(Collectors.toList())); return this; @@ -370,14 +370,14 @@ public Builder addInputFile(InputFile inputFile) { return this; } - /** TODO fix documentation after addition of InputFilesToJoin + /** TODO fix documentation after addition of InputFilesToJoinColumns * Add an input file to read from. * * @param inputFiles input file to read from * @return self */ - public Builder addInputFilesToJoin(List inputFiles) { - this.inputFilesToJoin.add(inputFiles); + public Builder addInputFilesToJoinColumns(List inputFiles) { + this.inputFilesToJoinColumns.add(inputFiles); return this; } @@ -404,19 +404,19 @@ public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); Preconditions.checkArgument( - inputFilesToJoin.stream().allMatch(x -> x != null && !x.isEmpty()), + inputFilesToJoinColumns.stream().allMatch(x -> x != null && !x.isEmpty()), "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); Preconditions.checkArgument( - inputFilesToJoin.isEmpty() || pruneColumns == null, + inputFilesToJoinColumns.isEmpty() || pruneColumns == null, "Right side Input files join functionality does not yet support column pruning"); Preconditions.checkArgument( - inputFilesToJoin.isEmpty() || maskColumns == null, + inputFilesToJoinColumns.isEmpty() || maskColumns == null, "Right side Input files join functionality does not yet support column masking"); Preconditions.checkArgument( - inputFilesToJoin.isEmpty() || encryptColumns == null, + inputFilesToJoinColumns.isEmpty() || encryptColumns == null, "Right side Input files join functionality does not yet support column encryption"); Preconditions.checkArgument( - inputFilesToJoin.isEmpty() || newCodecName == null, + inputFilesToJoinColumns.isEmpty() || newCodecName == null, "Right side Input files join functionality does not yet support codec changing"); if (pruneColumns != null) { @@ -450,7 +450,7 @@ public RewriteOptions build() { return new RewriteOptions( conf, inputFiles, - inputFilesToJoin, + inputFilesToJoinColumns, outputFile, pruneColumns, newCodecName, diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 946509f98b..bd0fa6b168 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -111,7 +111,7 @@ public class ParquetRewriterTest { private final boolean usingHadoop; private List inputFiles = null; - private List> inputFilesToJoin = null; + private List> inputFilesToJoinColumns = null; private String outputFile = null; private ParquetRewriter rewriter = null; @@ -179,7 +179,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E @Before public void setUp() { outputFile = TestFileBuilder.createTempFile("test"); - inputFilesToJoin = new ArrayList<>(); + inputFilesToJoinColumns = new ArrayList<>(); } @Test @@ -736,7 +736,7 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); - List> inputPathsR = inputFilesToJoin.stream() + List> inputPathsR = inputFilesToJoinColumns.stream() .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) .collect(Collectors.toList()); RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); @@ -810,7 +810,7 @@ private void testThreeInputsDifferentRowGroupSize() throws IOException { .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - inputFilesToJoin = Lists.newArrayList(Lists.newArrayList( + inputFilesToJoinColumns = Lists.newArrayList(Lists.newArrayList( Lists.newArrayList(new TestFileBuilder(conf, createSchemaR1()) .withNumRecord(numRecord) .withRowGroupSize(7_000_000) @@ -896,7 +896,7 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } - List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesToJoin.stream()) + List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesToJoinColumns.stream()) .map(x -> x.stream() .flatMap(y -> Arrays.stream(y.getFileContent())) .collect(Collectors.toList())) @@ -1119,7 +1119,7 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); List inFiles = Stream.concat( - inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) + inputFiles.stream(), inputFilesToJoinColumns.stream().flatMap(Collection::stream)) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); @@ -1164,7 +1164,7 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters(FileDecryptionProperties fileDecryptionProperties) throws Exception { Map> inputBloomFilters = new HashMap<>(); - List files = Stream.concat(Stream.of(inputFiles), inputFilesToJoin.stream()) + List files = Stream.concat(Stream.of(inputFiles), inputFilesToJoinColumns.stream()) .flatMap(Collection::stream) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : files) { From 1e7998a1a1d6abc4ed299c1678bfd4c69093b9d7 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 10 Apr 2024 20:26:04 -0700 Subject: [PATCH 20/57] add getParquetInputFiles listing to the rewrite start logging --- .../apache/parquet/hadoop/rewrite/ParquetRewriter.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 3a0a8ca8da..dc6110d648 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -133,7 +133,13 @@ public ParquetRewriter(RewriteOptions options) throws IOException { .collect(Collectors.toList()); ensureSameSchema(inputFiles); inputFilesToJoin.forEach(this::ensureSameSchema); - LOG.info("Start rewriting {} input file(s) {} to {}", inputFiles.size(), options.getParquetInputFiles(), out); + LOG.info( + "Start rewriting {} input file(s) {} with {} groups of input file(s) to join {} to {}", + inputFilesToJoin.size(), + options.getParquetInputFilesToJoinColumns(), + inputFiles.size(), + options.getParquetInputFiles(), + out); extraMetaData.put( ORIGINAL_CREATED_BY_KEY, From 2ee9b4073a43fc7b162feed03ea7552b74e05e48 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Apr 2024 13:19:57 -0700 Subject: [PATCH 21/57] redesign file joiner in ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 525 ++++++------------ .../hadoop/rewrite/RewriteOptions.java | 80 ++- .../hadoop/rewrite/ParquetRewriterTest.java | 95 +--- 3 files changed, 261 insertions(+), 439 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index dc6110d648..88ec39fdf5 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -26,8 +26,8 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Optional; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -66,7 +66,6 @@ import org.apache.parquet.hadoop.CodecFactory; import org.apache.parquet.hadoop.ColumnChunkPageWriteStore; import org.apache.parquet.hadoop.IndexCache; -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; @@ -80,7 +79,6 @@ import org.apache.parquet.io.InputFile; import org.apache.parquet.io.OutputFile; import org.apache.parquet.io.ParquetEncodingException; -import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.Converter; import org.apache.parquet.io.api.GroupConverter; import org.apache.parquet.io.api.PrimitiveConverter; @@ -111,28 +109,22 @@ public class ParquetRewriter implements Closeable { private int numBlocksRewritten = 0; // Reader and relevant states of the in-processing input file private final Queue inputFiles = new LinkedList<>(); - private final List columnWritersR = new ArrayList<>(); - // Schema of input files (should be the same) and to write to the output file - private MessageType schema = null; - private final Map descriptorsMap; - // The reader for the current input file - private TransParquetFileReader reader = null; - // The metadata of current reader being processed - private ParquetMetadata meta = null; - // created_by information of current reader being processed - private String originalCreatedBy = ""; + private final Queue inputFilesToJoin = new LinkedList<>(); + private MessageType outSchema; // The index cache strategy private final IndexCache.CacheStrategy indexCacheStrategy; + private final boolean joinColumnsOverwrite; public ParquetRewriter(RewriteOptions options) throws IOException { + this.newCodecName = options.getNewCodecName(); + this.indexCacheStrategy = options.getIndexCacheStrategy(); + this.joinColumnsOverwrite = options.getJoinColumnsOverwrite(); ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - List> inputFilesToJoin = options.getParquetInputFilesToJoinColumns().stream() - .map(x -> getFileReaders(x, conf)) - .collect(Collectors.toList()); + inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoinColumns(), conf)); ensureSameSchema(inputFiles); - inputFilesToJoin.forEach(this::ensureSameSchema); + ensureSameSchema(inputFilesToJoin); LOG.info( "Start rewriting {} input file(s) {} with {} groups of input file(s) to join {} to {}", inputFilesToJoin.size(), @@ -141,79 +133,32 @@ public ParquetRewriter(RewriteOptions options) throws IOException { options.getParquetInputFiles(), out); + this.outSchema = getSchema(); + this.outSchema = pruneColumnsInSchema(outSchema, options.getPruneColumns()); extraMetaData.put( ORIGINAL_CREATED_BY_KEY, - Stream.concat(inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) .collect(Collectors.toSet()) .stream() .reduce((a, b) -> a + "\n" + b) .orElse("")); - Stream.concat(inputFiles.stream(), inputFilesToJoin.stream().flatMap(Collection::stream)) + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); - MessageType schemaL = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - List schemaR = inputFilesToJoin.stream() - .map(x -> x.peek().getFooter().getFileMetaData().getSchema()) - .collect(Collectors.toList()); - Map fieldNamesL = new LinkedHashMap<>(); - schemaL.getFields().forEach(x -> fieldNamesL.put(x.getName(), x)); - Map fieldNamesR = new LinkedHashMap<>(); - schemaR.stream().flatMap(x -> x.getFields().stream()).forEach(x -> { - if (fieldNamesR.containsKey(x.getName())) { - throw new IllegalArgumentException( - "Found a duplicated field `" + x.getName() + "` in the right side file groups!"); + if (!inputFilesToJoin.isEmpty()) { + List blocksRowCountsL = inputFiles.stream().flatMap(x -> + x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount) + ).collect(Collectors.toList()); + List blocksRowCountsR = inputFilesToJoin.stream().flatMap(x -> + x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount) + ).collect(Collectors.toList()); + if (!blocksRowCountsL.equals(blocksRowCountsR)) { + throw new IllegalArgumentException("The number of rows in each block must match! Left blocks row counts: " + + blocksRowCountsL + ", right blocks row counts" + blocksRowCountsR + "."); } - fieldNamesR.put(x.getName(), x); - }); - List fields = Stream.concat( - fieldNamesL.values().stream() - .map(x -> fieldNamesR.getOrDefault( - x.getName(), x)), // take a field on the right if we can - fieldNamesR.values().stream() - .filter(x -> !fieldNamesL.containsKey( - x.getName())) // takes fields on the right if it was not present on the left - ) - .collect(Collectors.toList()); - schema = new MessageType(schemaL.getName(), fields); - - newCodecName = options.getNewCodecName(); - List pruneColumns = options.getPruneColumns(); - // Prune columns if specified - if (pruneColumns != null && !pruneColumns.isEmpty()) { - List paths = new ArrayList<>(); - getPaths(schema, paths, null); - for (String col : pruneColumns) { - if (!paths.contains(col)) { - LOG.warn("Input column name {} doesn't show up in the schema", col); - } - } - - Set prunePaths = convertToColumnPaths(pruneColumns); - schema = pruneColumnsInSchema(schema, prunePaths); - } - - if (inputFilesToJoin.isEmpty()) { - this.descriptorsMap = - schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - } else { // TODO: describe in documentation that only top level column can be overwritten - this.descriptorsMap = schemaL.getColumns().stream() - .filter(x -> x.getPath().length == 0 || !fieldNamesR.containsKey(x.getPath()[0])) - .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); } - long rowCountL = - inputFiles.stream().mapToLong(ParquetFileReader::getRecordCount).sum(); - inputFilesToJoin.stream() - .map(x -> - x.stream().mapToLong(ParquetFileReader::getRecordCount).sum()) - .forEach(rowCountR -> { - if (rowCountL != rowCountR) { - throw new IllegalArgumentException("The number of records on the left(" + rowCountL - + ") and on the right(" + rowCountR + ") don't match!"); - } - }); - if (options.getMaskColumns() != null) { this.maskColumns = new HashMap<>(); for (Map.Entry col : options.getMaskColumns().entrySet()) { @@ -226,12 +171,10 @@ public ParquetRewriter(RewriteOptions options) throws IOException { this.encryptMode = true; } - this.indexCacheStrategy = options.getIndexCacheStrategy(); - ParquetFileWriter.Mode writerMode = ParquetFileWriter.Mode.CREATE; writer = new ParquetFileWriter( out, - schema, + outSchema, writerMode, DEFAULT_BLOCK_SIZE, MAX_PADDING_SIZE_DEFAULT, @@ -241,8 +184,24 @@ public ParquetRewriter(RewriteOptions options) throws IOException { options.getFileEncryptionProperties()); writer.start(); - for (Queue inFiles : inputFilesToJoin) { - this.columnWritersR.add(new RightColumnWriter(inFiles, this)); + } + + private MessageType getSchema() { + MessageType schemaMain = inputFiles.peek().getFooter().getFileMetaData().getSchema(); + if (inputFilesToJoin.isEmpty()) { + return schemaMain; + } else { + Map fieldNames = new LinkedHashMap<>(); + schemaMain.getFields().forEach(x -> fieldNames.put(x.getName(), x)); + inputFilesToJoin.peek().getFooter().getFileMetaData().getSchema().getFields().forEach(x -> { + if (!fieldNames.containsKey(x.getName()) || joinColumnsOverwrite) { + fieldNames.put(x.getName(), x); + } + }); + return new MessageType( + schemaMain.getName(), + new ArrayList<>(fieldNames.values()) + ); } } @@ -251,15 +210,15 @@ public ParquetRewriter( TransParquetFileReader reader, ParquetFileWriter writer, ParquetMetadata meta, - MessageType schema, + MessageType outSchema, String originalCreatedBy, CompressionCodecName codecName, List maskColumns, MaskMode maskMode) { this.writer = writer; - this.schema = schema; - this.descriptorsMap = - schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); + this.outSchema = outSchema; +// this.descriptorsMap = +// schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); this.newCodecName = codecName; originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; extraMetaData.putAll(meta.getFileMetaData().getKeyValueMetaData()); @@ -272,6 +231,7 @@ public ParquetRewriter( } this.inputFiles.add(reader); this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; + this.joinColumnsOverwrite = false; } private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { @@ -315,101 +275,138 @@ public void close() throws IOException { } public void processBlocks() throws IOException { + TransParquetFileReader readerR = inputFilesToJoin.peek(); + IndexCache indexCacheR = null; + int blockIdxR = -1; + while (!inputFiles.isEmpty()) { - reader = inputFiles.poll(); - meta = reader.getFooter(); - originalCreatedBy = meta.getFileMetaData().getCreatedBy(); + TransParquetFileReader reader = inputFiles.poll(); LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); - IndexCache indexCache = IndexCache.create(reader, descriptorsMap.keySet(), indexCacheStrategy, true); - processBlocksFromReader(indexCache); - indexCache.clean(); - LOG.info("Finish rewriting input file: {}", reader.getFile()); - } - } - - private void processBlocksFromReader(IndexCache indexCache) throws IOException { - for (int blockId = 0; blockId < meta.getBlocks().size(); blockId++) { - BlockMetaData blockMetaData = meta.getBlocks().get(blockId); - writer.startBlock(blockMetaData.getRowCount()); - indexCache.setBlockMetadata(blockMetaData); - List columnsInOrder = blockMetaData.getColumns(); - for (int i = 0, columnId = 0; i < columnsInOrder.size(); i++) { - ColumnChunkMetaData chunk = columnsInOrder.get(i); - ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); - - // This column has been pruned. - if (descriptor == null) { - continue; - } - - // If a column is encrypted, we simply throw exception. - // Later we can add a feature to trans-encrypt it with different keys - if (chunk.isEncrypted()) { - throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted"); + ParquetMetadata meta = reader.getFooter(); + Set columnPaths = meta.getFileMetaData().getSchema().getColumns().stream() + .map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + IndexCache indexCache = IndexCache.create(reader, columnPaths, indexCacheStrategy, true); + + for (int blockIdx = 0; blockIdx < meta.getBlocks().size(); blockIdx++) { + BlockMetaData blockMetaData = meta.getBlocks().get(blockIdx); + writer.startBlock(blockMetaData.getRowCount()); + indexCache.setBlockMetadata(blockMetaData); + Map pathToChunk = blockMetaData.getColumns().stream() + .collect(Collectors.toMap(x -> x.getPath(), x -> x)); + + if (readerR != null && (blockIdxR == -1 || ++blockIdxR == readerR.getFooter().getBlocks().size())) { + blockIdxR = 0; + readerR = inputFilesToJoin.poll(); + Set columnPathsR = readerR.getFileMetaData().getSchema().getColumns().stream() + .map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + if (indexCacheR != null) { + indexCacheR.clean(); + } + indexCacheR = IndexCache.create(readerR, columnPathsR, indexCacheStrategy, true); + indexCacheR.setBlockMetadata(readerR.getFooter().getBlocks().get(blockIdxR)); + } else { + blockIdxR++; } - reader.setStreamPosition(chunk.getStartingPos()); - CompressionCodecName newCodecName = this.newCodecName == null ? chunk.getCodec() : this.newCodecName; - boolean encryptColumn = - encryptMode && encryptColumns != null && encryptColumns.contains(chunk.getPath()); - - if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) { - // Mask column and compress it again. - MaskMode maskMode = maskColumns.get(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"); + for (int outColumnIdx = 0; outColumnIdx < outSchema.getColumns().size(); outColumnIdx++) { + ColumnDescriptor descriptor = outSchema.getColumns().get(outColumnIdx); + ColumnPath colPath = ColumnPath.get(descriptor.getPath()); + ColumnChunkMetaData chunk = pathToChunk.get(colPath); + + if (readerR != null) { + Optional chunkR = readerR.getFooter().getBlocks().get(blockIdxR).getColumns() + .stream().filter(x -> x.getPath().equals(colPath)).findFirst(); + if (chunkR.isPresent() && (joinColumnsOverwrite || !columnPaths.contains(colPath))) { + processBlock(readerR, blockIdxR, outColumnIdx, indexCacheR, chunkR.get()); + } else { + processBlock(reader, blockIdx, outColumnIdx, indexCache, chunk); } - nullifyColumn(blockId, descriptor, chunk, writer, schema, newCodecName, encryptColumn); } else { - throw new UnsupportedOperationException("Only nullify is supported for now"); - } - } else if (encryptMode || this.newCodecName != null) { - // Prepare encryption context - ColumnChunkEncryptorRunTime columnChunkEncryptorRunTime = null; - if (encryptMode) { - columnChunkEncryptorRunTime = new ColumnChunkEncryptorRunTime( - writer.getEncryptor(), chunk, numBlocksRewritten, columnId); + processBlock(reader, blockIdx, outColumnIdx, indexCache, chunk); } - // Translate compression and/or encryption - writer.startColumn(descriptor, chunk.getValueCount(), newCodecName); - processChunk( - blockMetaData.getRowCount(), - chunk, - newCodecName, - columnChunkEncryptorRunTime, - encryptColumn, - indexCache.getBloomFilter(chunk), - indexCache.getColumnIndex(chunk), - indexCache.getOffsetIndex(chunk)); - writer.endColumn(); - } else { - // Nothing changed, simply copy the binary data. - BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); - ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); - OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk( - descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } - columnId++; + writer.endBlock(); + indexCache.clean(); + numBlocksRewritten++; } - // Writing extra columns - for (RightColumnWriter columnWriter : columnWritersR) { - columnWriter.writeRows(numBlocksRewritten, blockMetaData.getRowCount()); + indexCache.clean(); + LOG.info("Finish rewriting input file: {}", reader.getFile()); + + } + } + + private void processBlocksFromReader(TransParquetFileReader reader) throws IOException { + } + + private void processBlock( + TransParquetFileReader reader, + int blockIdx, + int outColumnIdx, + IndexCache indexCache, + ColumnChunkMetaData chunk) throws IOException { + if (chunk.isEncrypted()) { + throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted"); + } + ColumnDescriptor descriptor = outSchema.getColumns().get(outColumnIdx); + BlockMetaData blockMetaData = reader.getFooter().getBlocks().get(blockIdx); + String originalCreatedBy = reader.getFileMetaData().getCreatedBy(); + + reader.setStreamPosition(chunk.getStartingPos()); + CompressionCodecName newCodecName = this.newCodecName == null ? chunk.getCodec() : this.newCodecName; + boolean encryptColumn = + encryptMode && encryptColumns != null && encryptColumns.contains(chunk.getPath()); + + if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) { + // Mask column and compress it again. + MaskMode maskMode = maskColumns.get(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(reader, blockIdx, descriptor, chunk, writer, outSchema, newCodecName, encryptColumn, originalCreatedBy); + } else { + throw new UnsupportedOperationException("Only nullify is supported for now"); + } + } else if (encryptMode || this.newCodecName != null) { + // Prepare encryption context + ColumnChunkEncryptorRunTime columnChunkEncryptorRunTime = null; + if (encryptMode) { + columnChunkEncryptorRunTime = new ColumnChunkEncryptorRunTime( + writer.getEncryptor(), chunk, numBlocksRewritten, outColumnIdx); } - writer.endBlock(); - numBlocksRewritten++; + // Translate compression and/or encryption + writer.startColumn(descriptor, chunk.getValueCount(), newCodecName); + processChunk( + reader, + blockMetaData.getRowCount(), + chunk, + newCodecName, + columnChunkEncryptorRunTime, + encryptColumn, + indexCache.getBloomFilter(chunk), + indexCache.getColumnIndex(chunk), + indexCache.getOffsetIndex(chunk), + originalCreatedBy); + writer.endColumn(); + } else { + // Nothing changed, simply copy the binary data. + BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); + ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); + OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); + writer.appendColumnChunk( + descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } } private void processChunk( + TransParquetFileReader reader, long blockRowCount, ColumnChunkMetaData chunk, CompressionCodecName newCodecName, @@ -417,7 +414,8 @@ private void processChunk( boolean encryptColumn, BloomFilter bloomFilter, ColumnIndex columnIndex, - OffsetIndex offsetIndex) + OffsetIndex offsetIndex, + String originalCreatedBy) throws IOException { CompressionCodecFactory codecFactory = HadoopCodecs.newFactory(0); CompressionCodecFactory.BytesInputDecompressor decompressor = null; @@ -720,11 +718,24 @@ private void getPaths(GroupType schema, List paths, String parent) { } } - private MessageType pruneColumnsInSchema(MessageType schema, Set prunePaths) { - List fields = schema.getFields(); - List currentPath = new ArrayList<>(); - List prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths); - return new MessageType(schema.getName(), prunedFields); + private MessageType pruneColumnsInSchema(MessageType schema, List pruneColumns) { + if (pruneColumns == null || pruneColumns.isEmpty()) { + return schema; + } else { + List paths = new ArrayList<>(); + getPaths(schema, paths, null); + for (String col : pruneColumns) { + if (!paths.contains(col)) { + LOG.warn("Input column name {} doesn't show up in the schema", col); + } + } + Set prunePaths = convertToColumnPaths(pruneColumns); + + List fields = schema.getFields(); + List currentPath = new ArrayList<>(); + List prunedFields = pruneColumnsInFields(fields, currentPath, prunePaths); + return new MessageType(schema.getName(), prunedFields); + } } private List pruneColumnsInFields(List fields, List currentPath, Set prunePaths) { @@ -768,13 +779,15 @@ private Set convertToColumnPaths(List cols) { } private void nullifyColumn( + TransParquetFileReader reader, int blockIndex, ColumnDescriptor descriptor, ColumnChunkMetaData chunk, ParquetFileWriter writer, MessageType schema, CompressionCodecName newCodecName, - boolean encryptColumn) + boolean encryptColumn, + String originalCreatedBy) throws IOException { if (encryptColumn) { Preconditions.checkArgument(writer.getEncryptor() != null, "Missing encryptor"); @@ -962,174 +975,4 @@ public byte[] getDictPageAAD() { } } - private static class RightColumnWriter { - private final Queue inputFiles; - private final ParquetRewriter parquetRewriter; - private final ParquetFileWriter writer; - private final MessageType schema; - private final Map descriptorsMap; - private final Map colReaders = new HashMap<>(); - private final Map cPageStores = new HashMap<>(); - private final Map cStores = new HashMap<>(); - private final Map cWriters = new HashMap<>(); - private int rowGroupIdxL = 0; // index of the rowGroup of the current file on the left - private int rowGroupIdxR = 0; // index of the rowGroup of the current file on the right - private int writtenFromBlock = 0; - - public RightColumnWriter(Queue inputFiles, ParquetRewriter parquetRewriter) - throws IOException { - this.inputFiles = inputFiles; - this.parquetRewriter = parquetRewriter; - this.writer = parquetRewriter.writer; - this.schema = inputFiles.peek().getFooter().getFileMetaData().getSchema(); - this.descriptorsMap = this.schema.getColumns().stream() - .collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); - initReaders(); - initWriters(); - } - - public void writeRows(int rowGroupIdx, long rowsToWrite) throws IOException { - if (rowGroupIdxL > rowGroupIdx) { - throw new IOException("A row group index decrease is determined in RightColumnWriter! Current index: " - + rowGroupIdxL + ", new index: " + rowGroupIdx); - } - if (rowGroupIdxL != rowGroupIdx) { - rowGroupIdxL = rowGroupIdx; - flushWriters(); - initWriters(); - } - while (rowsToWrite > 0) { - List blocks = inputFiles.peek().getFooter().getBlocks(); - BlockMetaData block = blocks.get(rowGroupIdxR); - List chunks = block.getColumns(); - long leftInBlock = block.getRowCount() - writtenFromBlock; - long writeFromBlock = Math.min(rowsToWrite, leftInBlock); - for (ColumnChunkMetaData chunk : chunks) { - if (chunk.isEncrypted()) { - throw new IOException("Column " + chunk.getPath().toDotString() + " is encrypted"); - } - ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath()); - copyValues(descriptor, writeFromBlock); - } - rowsToWrite -= writeFromBlock; - writtenFromBlock += writeFromBlock; - if (rowsToWrite > 0 || (block.getRowCount() == writtenFromBlock)) { - rowGroupIdxR++; - if (rowGroupIdxR == blocks.size()) { - inputFiles.poll(); - rowGroupIdxR = 0; - } - writtenFromBlock = 0; - // this is called after all rows are processed - initReaders(); - } - } - flushWriters(); - } - - private void flushWriters() throws IOException { - cStores.values().forEach(cStore -> { - cStore.flush(); - cStore.close(); - }); - cWriters.values().forEach(ColumnWriter::close); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { - if (cPageStores.containsKey(descriptor)) - cPageStores.get(descriptor).flushToFileWriter(writer); - } - cStores.clear(); - cWriters.clear(); - cPageStores.clear(); - } - - private void initWriters() { - if (!inputFiles.isEmpty()) { - List blocks = inputFiles.peek().getFooter().getBlocks(); - descriptorsMap.forEach((columnPath, descriptor) -> { - ColumnChunkMetaData chunk = blocks.get(rowGroupIdxR).getColumns().stream() - .filter(x -> x.getPath() == columnPath) - .findFirst() - .orElseThrow(() -> new IllegalStateException( - "Could not find column [" + columnPath.toDotString() + "].")); - int bloomFilterLength = chunk.getBloomFilterLength(); - ParquetProperties.WriterVersion writerVersion = - chunk.getEncodingStats().usesV2Pages() - ? ParquetProperties.WriterVersion.PARQUET_2_0 - : ParquetProperties.WriterVersion.PARQUET_1_0; - ParquetProperties props = ParquetProperties.builder() - .withWriterVersion(writerVersion) - .withBloomFilterEnabled(bloomFilterLength > 0) - .build(); - CodecFactory codecFactory = new CodecFactory(new Configuration(), props.getPageSizeThreshold()); - CompressionCodecFactory.BytesInputCompressor compressor = - codecFactory.getCompressor(chunk.getCodec()); - - MessageType columnSchema = parquetRewriter.newSchema(schema, descriptor); - ColumnChunkPageWriteStore cPageStore = new ColumnChunkPageWriteStore( - compressor, - columnSchema, - props.getAllocator(), - props.getColumnIndexTruncateLength(), - props.getPageWriteChecksumEnabled(), - writer.getEncryptor(), - rowGroupIdxL); - ColumnWriteStore cwStore = props.newColumnWriteStore(columnSchema, cPageStore, cPageStore); - ColumnWriter cWriter = cwStore.getColumnWriter(descriptor); - cPageStores.put(descriptor, cPageStore); - cStores.put(descriptor, cwStore); - cWriters.put(descriptor, cWriter); - }); - } - } - - private void initReaders() throws IOException { - if (!inputFiles.isEmpty()) { - TransParquetFileReader reader = inputFiles.peek(); - PageReadStore pageReadStore = reader.readRowGroup(rowGroupIdxR); - String createdBy = reader.getFooter().getFileMetaData().getCreatedBy(); - ColumnReadStoreImpl crStore = new ColumnReadStoreImpl( - pageReadStore, new ParquetRewriter.DummyGroupConverter(), schema, createdBy); - for (ColumnDescriptor descriptor : descriptorsMap.values()) { - ColumnReader cReader = crStore.getColumnReader(descriptor); - colReaders.put(descriptor, cReader); - } - } - } - - private void copyValues(ColumnDescriptor descriptor, long rowsToWrite) { - ColumnWriteStore cStore = cStores.get(descriptor); - ColumnWriter cWriter = cWriters.get(descriptor); - int dMax = descriptor.getMaxDefinitionLevel(); - Class columnType = descriptor.getPrimitiveType().getPrimitiveTypeName().javaType; - ColumnReader reader = colReaders.get(descriptor); - for (int i = 0; i < rowsToWrite; i++) { - int rlvl = reader.getCurrentRepetitionLevel(); - int dlvl = reader.getCurrentDefinitionLevel(); - do { - if (dlvl < dMax) { - cWriter.writeNull(rlvl, dlvl); - } else if (columnType == Integer.TYPE) { - cWriter.write(reader.getInteger(), rlvl, dlvl); - } else if (columnType == Long.TYPE) { - cWriter.write(reader.getLong(), rlvl, dlvl); - } else if (columnType == Float.TYPE) { - cWriter.write(reader.getFloat(), rlvl, dlvl); - } else if (columnType == Double.TYPE) { - cWriter.write(reader.getDouble(), rlvl, dlvl); - } else if (columnType == Binary.class) { - cWriter.write(reader.getBinary(), rlvl, dlvl); - } else if (columnType == Boolean.TYPE) { - cWriter.write(reader.getBoolean(), rlvl, dlvl); - } else { - throw new UnsupportedOperationException( - String.format("Unsupported column java class: %s", columnType.toString())); - } - reader.consume(); - rlvl = reader.getCurrentRepetitionLevel(); - dlvl = reader.getCurrentDefinitionLevel(); - } while (rlvl > 0); - cStore.endRecord(); - } - } - } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 7ba527e3f0..b43806eea7 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -52,7 +52,7 @@ public class RewriteOptions { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesToJoinColumns; + private final List inputFilesToJoinColumns; private final OutputFile outputFile; private final List pruneColumns; private final CompressionCodecName newCodecName; @@ -60,18 +60,20 @@ public class RewriteOptions { private final List encryptColumns; private final FileEncryptionProperties fileEncryptionProperties; private final IndexCache.CacheStrategy indexCacheStrategy; + private final boolean joinColumnsOverwrite; private RewriteOptions( ParquetConfiguration conf, List inputFiles, - List> inputFilesToJoinColumns, + List inputFilesToJoinColumns, OutputFile outputFile, List pruneColumns, CompressionCodecName newCodecName, Map maskColumns, List encryptColumns, FileEncryptionProperties fileEncryptionProperties, - IndexCache.CacheStrategy indexCacheStrategy) { + IndexCache.CacheStrategy indexCacheStrategy, + boolean joinColumnsOverwrite) { this.conf = conf; this.inputFiles = inputFiles; this.inputFilesToJoinColumns = inputFilesToJoinColumns; @@ -82,6 +84,7 @@ private RewriteOptions( this.encryptColumns = encryptColumns; this.fileEncryptionProperties = fileEncryptionProperties; this.indexCacheStrategy = indexCacheStrategy; + this.joinColumnsOverwrite = joinColumnsOverwrite; } /** @@ -135,7 +138,7 @@ public List getParquetInputFiles() { * * @return a {@link List} of the associated right {@link InputFile}s */ - public List> getParquetInputFilesToJoinColumns() { + public List getParquetInputFilesToJoinColumns() { return inputFilesToJoinColumns; } @@ -186,11 +189,15 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { return indexCacheStrategy; } + public boolean getJoinColumnsOverwrite() { + return joinColumnsOverwrite; + } + // Builder to create a RewriterOptions. public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; - private final List> inputFilesToJoinColumns = new ArrayList<>(); + private final List inputFilesToJoinColumns; private final OutputFile outputFile; private List pruneColumns; private CompressionCodecName newCodecName; @@ -198,6 +205,7 @@ public static class Builder { private List encryptColumns; private FileEncryptionProperties fileEncryptionProperties; private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; + private boolean joinColumnsOverwrite = false; /** * Create a builder to create a RewriterOptions. @@ -245,6 +253,7 @@ public Builder(Configuration conf, List inputFiles, Path outputFile) { for (Path inputFile : inputFiles) { this.inputFiles.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); } + this.inputFilesToJoinColumns = new ArrayList<>(); this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } @@ -266,6 +275,27 @@ public Builder(Configuration conf, List inputFiles, Path outputFile) { public Builder(ParquetConfiguration conf, List inputFiles, OutputFile outputFile) { this.conf = conf; this.inputFiles = inputFiles; + this.inputFilesToJoinColumns = new ArrayList<>(); + this.outputFile = outputFile; + } + + public Builder(Configuration conf, List inputFiles, List inputFilesToJoinColumns, Path outputFile) { + this.conf = new HadoopParquetConfiguration(conf); + this.inputFiles = new ArrayList<>(inputFiles.size()); + for (Path inputFile : inputFiles) { + this.inputFiles.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); + } + this.inputFilesToJoinColumns = new ArrayList<>(inputFilesToJoinColumns.size()); + for (Path inputFile : inputFilesToJoinColumns) { + this.inputFilesToJoinColumns.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); + } + this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); + } + + public Builder(ParquetConfiguration conf, List inputFiles, List inputFilesToJoinColumns, OutputFile outputFile) { + this.conf = conf; + this.inputFiles = inputFiles; + this.inputFilesToJoinColumns = inputFilesToJoinColumns; this.outputFile = outputFile; } @@ -349,13 +379,12 @@ public Builder addInputFile(Path path) { /** TODO fix documentation after addition of InputFilesToJoinColumns * Add an input file to read from. * - * @param paths input file path to read from + * @param path input file path to read from * @return self */ - public Builder addInputPathsR(List paths) { - this.inputFilesToJoinColumns.add(paths.stream() - .map(x -> HadoopInputFile.fromPathUnchecked(x, ConfigurationUtil.createHadoopConfiguration(conf))) - .collect(Collectors.toList())); + public Builder addInputFileToJoinColumns(Path path) { + this.inputFilesToJoinColumns.add( + HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); return this; } @@ -373,11 +402,11 @@ public Builder addInputFile(InputFile inputFile) { /** TODO fix documentation after addition of InputFilesToJoinColumns * Add an input file to read from. * - * @param inputFiles input file to read from + * @param inputFile input file to read from * @return self */ - public Builder addInputFilesToJoinColumns(List inputFiles) { - this.inputFilesToJoinColumns.add(inputFiles); + public Builder addInputFilesToJoinColumns(InputFile inputFile) { + this.inputFilesToJoinColumns.add(inputFile); return this; } @@ -395,6 +424,14 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { return this; } + + /** TODO fix documentation after addition of InputFilesToJoinColumns + */ + public Builder joinColumnsOverwrite(boolean joinColumnsOverwrite) { + this.joinColumnsOverwrite = joinColumnsOverwrite; + return this; + } + /** * Build the RewriterOptions. * @@ -404,20 +441,8 @@ public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); Preconditions.checkArgument( - inputFilesToJoinColumns.stream().allMatch(x -> x != null && !x.isEmpty()), + inputFiles != null && !inputFiles.isEmpty(), "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); - Preconditions.checkArgument( - inputFilesToJoinColumns.isEmpty() || pruneColumns == null, - "Right side Input files join functionality does not yet support column pruning"); - Preconditions.checkArgument( - inputFilesToJoinColumns.isEmpty() || maskColumns == null, - "Right side Input files join functionality does not yet support column masking"); - Preconditions.checkArgument( - inputFilesToJoinColumns.isEmpty() || encryptColumns == null, - "Right side Input files join functionality does not yet support column encryption"); - Preconditions.checkArgument( - inputFilesToJoinColumns.isEmpty() || newCodecName == null, - "Right side Input files join functionality does not yet support codec changing"); if (pruneColumns != null) { if (maskColumns != null) { @@ -457,7 +482,8 @@ public RewriteOptions build() { maskColumns, encryptColumns, fileEncryptionProperties, - indexCacheStrategy); + indexCacheStrategy, + joinColumnsOverwrite); } } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index bd0fa6b168..1932d8f335 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -111,16 +111,16 @@ public class ParquetRewriterTest { private final boolean usingHadoop; private List inputFiles = null; - private List> inputFilesToJoinColumns = null; + private List inputFilesToJoinColumns = null; private String outputFile = null; private ParquetRewriter rewriter = null; @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { return new Object[][] { - {"v1", "NONE", true}, - {"v1", "PREFETCH_BLOCK", true}, - {"v2", "PREFETCH_BLOCK", true}, + // {"v1", "NONE", true}, + // {"v1", "PREFETCH_BLOCK", true}, + // {"v2", "PREFETCH_BLOCK", true}, {"v2", "PREFETCH_BLOCK", false} }; } @@ -736,8 +736,7 @@ public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); - List> inputPathsR = inputFilesToJoinColumns.stream() - .map(x -> x.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList())) + List inputPathsR = inputFilesToJoinColumns.stream().map(y -> new Path(y.getFileName())) .collect(Collectors.toList()); RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); @@ -753,19 +752,11 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) Map> inputBloomFilters = allInputBloomFilters(null); Map> outputBloomFilters = allOutputBloomFilters(null); - Set schemaR1Columns = createSchemaR1().getColumns().stream() + Set schemaRColumns = createSchemaR().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); - Set schemaR2Columns = createSchemaR2().getColumns().stream() - .map(x -> ColumnPath.get(x.getPath())) - .collect(Collectors.toSet()); - Set r1BloomFilters = outputBloomFilters.keySet().stream() - .filter(schemaR1Columns::contains) - .collect(Collectors.toSet()); - Set r2withBloomFilters = outputBloomFilters.keySet().stream() - .filter(schemaR2Columns::contains) - .collect(Collectors.toSet()); - Set rBloomFilters = Stream.concat(r1BloomFilters.stream(), r2withBloomFilters.stream()) + Set rBloomFilters = outputBloomFilters.keySet().stream() + .filter(schemaRColumns::contains) .collect(Collectors.toSet()); // TODO potentially too many checks, might need to be split into multiple tests @@ -797,50 +788,16 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) private void testThreeInputsDifferentRowGroupSize() throws IOException { inputFiles = Lists.newArrayList( new TestFileBuilder(conf, createSchemaL()) - .withNumRecord(numRecord / 2) - .withRowGroupSize(5_000_000) - .withCodec("GZIP") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaL()) - .withNumRecord(numRecord - (numRecord / 2)) - .withRowGroupSize(6_000_000) .withCodec("GZIP") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - inputFilesToJoinColumns = Lists.newArrayList(Lists.newArrayList( - Lists.newArrayList(new TestFileBuilder(conf, createSchemaR1()) - .withNumRecord(numRecord) - .withRowGroupSize(7_000_000) - .withCodec("ZSTD") + inputFilesToJoinColumns = Lists.newArrayList( + new TestFileBuilder(conf, createSchemaR()) + .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) - .withBloomFilterEnabled(new String[] {"Links.Forward"}) - .build()), - Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(200_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord / 3) - .withRowGroupSize(300_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR2()) - .withNumRecord(numRecord - 2 * (numRecord / 3)) - .withRowGroupSize(400_000) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build()))); + .build()); } private MessageType createSchema() { @@ -868,9 +825,10 @@ private MessageType createSchemaL() { new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); } - private MessageType createSchemaR1() { + private MessageType createSchemaR() { return new MessageType( "schema", + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), new GroupType( OPTIONAL, "Links", @@ -878,10 +836,6 @@ private MessageType createSchemaR1() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } - private MessageType createSchemaR2() { - return new MessageType("schema", new PrimitiveType(REPEATED, FLOAT, "FloatFraction")); - } - private void validateColumnData( Set prunePaths, Set nullifiedPaths, FileDecryptionProperties fileDecryptionProperties) throws IOException { @@ -896,9 +850,8 @@ private void validateColumnData( totalRows += inputFile.getFileContent().length; } - List> fileContents = Stream.concat(Stream.of(inputFiles), inputFilesToJoinColumns.stream()) - .map(x -> x.stream() - .flatMap(y -> Arrays.stream(y.getFileContent())) + List> fileContents = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) + .map(x -> Arrays.stream(x.getFileContent()) .collect(Collectors.toList())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { @@ -1119,7 +1072,7 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); List inFiles = Stream.concat( - inputFiles.stream(), inputFilesToJoinColumns.stream().flatMap(Collection::stream)) + inputFiles.stream(), inputFilesToJoinColumns.stream()) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); @@ -1164,8 +1117,7 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters(FileDecryptionProperties fileDecryptionProperties) throws Exception { Map> inputBloomFilters = new HashMap<>(); - List files = Stream.concat(Stream.of(inputFiles), inputFilesToJoinColumns.stream()) - .flatMap(Collection::stream) + List files = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : files) { Map> bloomFilters = @@ -1214,20 +1166,21 @@ private RewriteOptions.Builder createBuilder(List inputPaths) throws IOExc return createBuilder(inputPaths, new ArrayList<>()); } - private RewriteOptions.Builder createBuilder(List inputPathsL, List> inputPathsR) + private RewriteOptions.Builder createBuilder(List inputPathsL, List inputPathsR) throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); - builder = new RewriteOptions.Builder(conf, inputPathsL, outputPath); - inputPathsR.forEach(builder::addInputPathsR); + builder = new RewriteOptions.Builder(conf, inputPathsL, inputPathsR, outputPath); } else { OutputFile outputPath = HadoopOutputFile.fromPath(new Path(outputFile), conf); List inputsL = inputPathsL.stream() .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) .collect(Collectors.toList()); - builder = new RewriteOptions.Builder(parquetConf, inputsL, outputPath); - inputPathsR.forEach(builder::addInputPathsR); + List inputsR = inputPathsR.stream() + .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) + .collect(Collectors.toList()); + builder = new RewriteOptions.Builder(parquetConf, inputsL, inputsR, outputPath); } return builder; } From db52c8516f47098c9ff9e51165c82ccc63d77d68 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Apr 2024 14:06:28 -0700 Subject: [PATCH 22/57] redesign file joiner in ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 2 -- .../hadoop/rewrite/ParquetRewriterTest.java | 28 ++++++++++++------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 88ec39fdf5..ae3dfb433c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -217,8 +217,6 @@ public ParquetRewriter( MaskMode maskMode) { this.writer = writer; this.outSchema = outSchema; -// this.descriptorsMap = -// schema.getColumns().stream().collect(Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x)); this.newCodecName = codecName; originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; extraMetaData.putAll(meta.getFileMetaData().getKeyValueMetaData()); diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 1932d8f335..e3f1507dcc 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -38,7 +38,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -731,14 +730,14 @@ private void testMultipleInputFilesSetup() throws IOException { } @Test - public void testStitchThreeInputsDifferentRowGroupSize() throws Exception { - testThreeInputsDifferentRowGroupSize(); + public void testStitchTwoInputs() throws Exception { + testTwoInputFileGroups(); List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); List inputPathsR = inputFilesToJoinColumns.stream().map(y -> new Path(y.getFileName())) .collect(Collectors.toList()); - RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR); + RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR, true); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); rewriter = new ParquetRewriter(options); @@ -770,7 +769,6 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) { add(CompressionCodecName.GZIP); add(CompressionCodecName.UNCOMPRESSED); - add(CompressionCodecName.ZSTD); } }, null); @@ -785,15 +783,25 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) }); } - private void testThreeInputsDifferentRowGroupSize() throws IOException { + private void testTwoInputFileGroups() throws IOException { inputFiles = Lists.newArrayList( new TestFileBuilder(conf, createSchemaL()) .withCodec("GZIP") + .withRowGroupSize(numRecord / 2) .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); inputFilesToJoinColumns = Lists.newArrayList( new TestFileBuilder(conf, createSchemaR()) + .withRowGroupSize(numRecord / 2) + .withNumRecord(numRecord / 2) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build(), + new TestFileBuilder(conf, createSchemaR()) + .withRowGroupSize(numRecord / 2) + .withNumRecord(numRecord / 2) .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) @@ -851,8 +859,7 @@ private void validateColumnData( } List> fileContents = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) - .map(x -> Arrays.stream(x.getFileContent()) - .collect(Collectors.toList())) + .map(x -> Arrays.stream(x.getFileContent()).collect(Collectors.toList())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { for (int i = fileContents.size() - 1; i >= 0; i--) { @@ -1163,10 +1170,10 @@ private Map> allBloomFilters( } private RewriteOptions.Builder createBuilder(List inputPaths) throws IOException { - return createBuilder(inputPaths, new ArrayList<>()); + return createBuilder(inputPaths, new ArrayList<>(), false); } - private RewriteOptions.Builder createBuilder(List inputPathsL, List inputPathsR) + private RewriteOptions.Builder createBuilder(List inputPathsL, List inputPathsR, boolean joinColumnsOverwrite) throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { @@ -1181,6 +1188,7 @@ private RewriteOptions.Builder createBuilder(List inputPathsL, List .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) .collect(Collectors.toList()); builder = new RewriteOptions.Builder(parquetConf, inputsL, inputsR, outputPath); + builder.joinColumnsOverwrite(joinColumnsOverwrite); } return builder; } From 9057e9162e06cf0fcec0f045924616f23e1d7047 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Apr 2024 15:13:29 -0700 Subject: [PATCH 23/57] redesign file joiner in ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 109 ++++++++++-------- .../hadoop/rewrite/RewriteOptions.java | 7 +- .../hadoop/rewrite/ParquetRewriterTest.java | 80 ++++++------- 3 files changed, 106 insertions(+), 90 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index ae3dfb433c..e9868dd823 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -26,7 +26,6 @@ import java.io.Closeable; import java.io.IOException; -import java.util.Optional; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -34,6 +33,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; @@ -147,15 +147,16 @@ public ParquetRewriter(RewriteOptions options) throws IOException { .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); if (!inputFilesToJoin.isEmpty()) { - List blocksRowCountsL = inputFiles.stream().flatMap(x -> - x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount) - ).collect(Collectors.toList()); - List blocksRowCountsR = inputFilesToJoin.stream().flatMap(x -> - x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount) - ).collect(Collectors.toList()); + List blocksRowCountsL = inputFiles.stream() + .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) + .collect(Collectors.toList()); + List blocksRowCountsR = inputFilesToJoin.stream() + .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) + .collect(Collectors.toList()); if (!blocksRowCountsL.equals(blocksRowCountsR)) { - throw new IllegalArgumentException("The number of rows in each block must match! Left blocks row counts: " - + blocksRowCountsL + ", right blocks row counts" + blocksRowCountsR + "."); + throw new IllegalArgumentException( + "The number of rows in each block must match! Left blocks row counts: " + blocksRowCountsL + + ", right blocks row counts" + blocksRowCountsR + "."); } } @@ -183,7 +184,6 @@ public ParquetRewriter(RewriteOptions options) throws IOException { ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED, options.getFileEncryptionProperties()); writer.start(); - } private MessageType getSchema() { @@ -193,15 +193,18 @@ private MessageType getSchema() { } else { Map fieldNames = new LinkedHashMap<>(); schemaMain.getFields().forEach(x -> fieldNames.put(x.getName(), x)); - inputFilesToJoin.peek().getFooter().getFileMetaData().getSchema().getFields().forEach(x -> { - if (!fieldNames.containsKey(x.getName()) || joinColumnsOverwrite) { - fieldNames.put(x.getName(), x); - } - }); - return new MessageType( - schemaMain.getName(), - new ArrayList<>(fieldNames.values()) - ); + inputFilesToJoin + .peek() + .getFooter() + .getFileMetaData() + .getSchema() + .getFields() + .forEach(x -> { + if (!fieldNames.containsKey(x.getName()) || joinColumnsOverwrite) { + fieldNames.put(x.getName(), x); + } + }); + return new MessageType(schemaMain.getName(), new ArrayList<>(fieldNames.values())); } } @@ -282,21 +285,26 @@ public void processBlocks() throws IOException { LOG.info("Rewriting input file: {}, remaining files: {}", reader.getFile(), inputFiles.size()); ParquetMetadata meta = reader.getFooter(); Set columnPaths = meta.getFileMetaData().getSchema().getColumns().stream() - .map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + .map(x -> ColumnPath.get(x.getPath())) + .collect(Collectors.toSet()); IndexCache indexCache = IndexCache.create(reader, columnPaths, indexCacheStrategy, true); for (int blockIdx = 0; blockIdx < meta.getBlocks().size(); blockIdx++) { BlockMetaData blockMetaData = meta.getBlocks().get(blockIdx); writer.startBlock(blockMetaData.getRowCount()); indexCache.setBlockMetadata(blockMetaData); - Map pathToChunk = blockMetaData.getColumns().stream() - .collect(Collectors.toMap(x -> x.getPath(), x -> x)); + Map pathToChunk = + blockMetaData.getColumns().stream().collect(Collectors.toMap(x -> x.getPath(), x -> x)); - if (readerR != null && (blockIdxR == -1 || ++blockIdxR == readerR.getFooter().getBlocks().size())) { + if (readerR != null + && (blockIdxR == -1 + || ++blockIdxR + == readerR.getFooter().getBlocks().size())) { blockIdxR = 0; readerR = inputFilesToJoin.poll(); Set columnPathsR = readerR.getFileMetaData().getSchema().getColumns().stream() - .map(x -> ColumnPath.get(x.getPath())).collect(Collectors.toSet()); + .map(x -> ColumnPath.get(x.getPath())) + .collect(Collectors.toSet()); if (indexCacheR != null) { indexCacheR.clean(); } @@ -307,22 +315,21 @@ public void processBlocks() throws IOException { } for (int outColumnIdx = 0; outColumnIdx < outSchema.getColumns().size(); outColumnIdx++) { - ColumnDescriptor descriptor = outSchema.getColumns().get(outColumnIdx); - ColumnPath colPath = ColumnPath.get(descriptor.getPath()); - ColumnChunkMetaData chunk = pathToChunk.get(colPath); - + ColumnPath colPath = ColumnPath.get( + outSchema.getColumns().get(outColumnIdx).getPath()); if (readerR != null) { - Optional chunkR = readerR.getFooter().getBlocks().get(blockIdxR).getColumns() - .stream().filter(x -> x.getPath().equals(colPath)).findFirst(); + Optional chunkR = + readerR.getFooter().getBlocks().get(blockIdxR).getColumns().stream() + .filter(x -> x.getPath().equals(colPath)) + .findFirst(); if (chunkR.isPresent() && (joinColumnsOverwrite || !columnPaths.contains(colPath))) { processBlock(readerR, blockIdxR, outColumnIdx, indexCacheR, chunkR.get()); } else { - processBlock(reader, blockIdx, outColumnIdx, indexCache, chunk); + processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } } else { - processBlock(reader, blockIdx, outColumnIdx, indexCache, chunk); + processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } - } writer.endBlock(); @@ -332,19 +339,18 @@ public void processBlocks() throws IOException { indexCache.clean(); LOG.info("Finish rewriting input file: {}", reader.getFile()); - } } - private void processBlocksFromReader(TransParquetFileReader reader) throws IOException { - } + private void processBlocksFromReader(TransParquetFileReader reader) throws IOException {} private void processBlock( TransParquetFileReader reader, int blockIdx, int outColumnIdx, IndexCache indexCache, - ColumnChunkMetaData chunk) throws IOException { + ColumnChunkMetaData chunk) + throws IOException { if (chunk.isEncrypted()) { throw new IOException("Column " + chunk.getPath().toDotString() + " is already encrypted"); } @@ -354,20 +360,27 @@ private void processBlock( reader.setStreamPosition(chunk.getStartingPos()); CompressionCodecName newCodecName = this.newCodecName == null ? chunk.getCodec() : this.newCodecName; - boolean encryptColumn = - encryptMode && encryptColumns != null && encryptColumns.contains(chunk.getPath()); + boolean encryptColumn = encryptMode && encryptColumns != null && encryptColumns.contains(chunk.getPath()); if (maskColumns != null && maskColumns.containsKey(chunk.getPath())) { // Mask column and compress it again. MaskMode maskMode = maskColumns.get(chunk.getPath()); if (maskMode.equals(MaskMode.NULLIFY)) { - Type.Repetition repetition = - descriptor.getPrimitiveType().getRepetition(); + Type.Repetition repetition = descriptor.getPrimitiveType().getRepetition(); if (repetition.equals(Type.Repetition.REQUIRED)) { - throw new IOException("Required column [" - + descriptor.getPrimitiveType().getName() + "] cannot be nullified"); + throw new IOException( + "Required column [" + descriptor.getPrimitiveType().getName() + "] cannot be nullified"); } - nullifyColumn(reader, blockIdx, descriptor, chunk, writer, outSchema, newCodecName, encryptColumn, originalCreatedBy); + nullifyColumn( + reader, + blockIdx, + descriptor, + chunk, + writer, + outSchema, + newCodecName, + encryptColumn, + originalCreatedBy); } else { throw new UnsupportedOperationException("Only nullify is supported for now"); } @@ -375,8 +388,8 @@ private void processBlock( // Prepare encryption context ColumnChunkEncryptorRunTime columnChunkEncryptorRunTime = null; if (encryptMode) { - columnChunkEncryptorRunTime = new ColumnChunkEncryptorRunTime( - writer.getEncryptor(), chunk, numBlocksRewritten, outColumnIdx); + columnChunkEncryptorRunTime = + new ColumnChunkEncryptorRunTime(writer.getEncryptor(), chunk, numBlocksRewritten, outColumnIdx); } // Translate compression and/or encryption @@ -398,8 +411,7 @@ private void processBlock( BloomFilter bloomFilter = indexCache.getBloomFilter(chunk); ColumnIndex columnIndex = indexCache.getColumnIndex(chunk); OffsetIndex offsetIndex = indexCache.getOffsetIndex(chunk); - writer.appendColumnChunk( - descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); + writer.appendColumnChunk(descriptor, reader.getStream(), chunk, bloomFilter, columnIndex, offsetIndex); } } @@ -972,5 +984,4 @@ public byte[] getDictPageAAD() { return this.dictPageAAD; } } - } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index b43806eea7..3b6fd73218 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -292,7 +292,11 @@ public Builder(Configuration conf, List inputFiles, List inputFilesT this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } - public Builder(ParquetConfiguration conf, List inputFiles, List inputFilesToJoinColumns, OutputFile outputFile) { + public Builder( + ParquetConfiguration conf, + List inputFiles, + List inputFilesToJoinColumns, + OutputFile outputFile) { this.conf = conf; this.inputFiles = inputFiles; this.inputFilesToJoinColumns = inputFilesToJoinColumns; @@ -424,7 +428,6 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { return this; } - /** TODO fix documentation after addition of InputFilesToJoinColumns */ public Builder joinColumnsOverwrite(boolean joinColumnsOverwrite) { diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index e3f1507dcc..b070f481b4 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -731,11 +731,15 @@ private void testMultipleInputFilesSetup() throws IOException { @Test public void testStitchTwoInputs() throws Exception { + // todo add a case when a number of main input files and joined input files do not match + // for that we need to have a capability of specifying the number of rows in the output file + // right now a withRowGroupSize doesn't allow to strictly set a number of rows in a group testTwoInputFileGroups(); List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); - List inputPathsR = inputFilesToJoinColumns.stream().map(y -> new Path(y.getFileName())) + List inputPathsR = inputFilesToJoinColumns.stream() + .map(y -> new Path(y.getFileName())) .collect(Collectors.toList()); RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR, true); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); @@ -784,28 +788,16 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) } private void testTwoInputFileGroups() throws IOException { - inputFiles = Lists.newArrayList( - new TestFileBuilder(conf, createSchemaL()) - .withCodec("GZIP") - .withRowGroupSize(numRecord / 2) - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build()); - inputFilesToJoinColumns = Lists.newArrayList( - new TestFileBuilder(conf, createSchemaR()) - .withRowGroupSize(numRecord / 2) - .withNumRecord(numRecord / 2) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build(), - new TestFileBuilder(conf, createSchemaR()) - .withRowGroupSize(numRecord / 2) - .withNumRecord(numRecord / 2) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build()); + inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaL()) + .withCodec("GZIP") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); + inputFilesToJoinColumns = Lists.newArrayList(new TestFileBuilder(conf, createSchemaR()) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); } private MessageType createSchema() { @@ -852,26 +844,37 @@ private void validateColumnData( .withDecryption(fileDecryptionProperties) .build(); - // Get total number of rows from input files - int totalRows = 0; - for (EncryptionTestFile inputFile : inputFiles) { - totalRows += inputFile.getFileContent().length; - } - - List> fileContents = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) + List> fileContents = inputFiles.stream() + .map(x -> Arrays.stream(x.getFileContent()).collect(Collectors.toList())) + .collect(Collectors.toList()); + List> fileContentsJoined = inputFilesToJoinColumns.stream() .map(x -> Arrays.stream(x.getFileContent()).collect(Collectors.toList())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { - for (int i = fileContents.size() - 1; i >= 0; i--) { - SimpleGroup expGroup = fileContents.get(i).get(rowIdx); - GroupType fileSchema = expGroup.getType(); - if (fileSchema.containsField(name)) { - return expGroup; + for (int i = 0; i < fileContents.size(); i++) { + if (rowIdx >= fileContents.get(i).size()) { + rowIdx -= fileContents.get(i).size(); + } else { + if (!fileContentsJoined.isEmpty()) { // todo check if joinColumnsOverwrite = true + SimpleGroup expGroup = fileContentsJoined.get(i).get(rowIdx); + GroupType fileSchema = expGroup.getType(); + if (fileSchema.containsField(name)) { + return expGroup; + } + } + SimpleGroup expGroup = fileContents.get(i).get(rowIdx); + GroupType fileSchema = expGroup.getType(); + if (fileSchema.containsField(name)) { + return expGroup; + } } } throw new IllegalStateException( "Group '" + name + "' at position " + rowIdx + " was not found in input files!"); }; + + int totalRows = + inputFiles.stream().mapToInt(x -> x.getFileContent().length).sum(); for (int i = 0; i < totalRows; i++) { Group group = reader.read(); assertNotNull(group); @@ -1078,8 +1081,7 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); - List inFiles = Stream.concat( - inputFiles.stream(), inputFilesToJoinColumns.stream()) + List inFiles = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) .collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); @@ -1173,8 +1175,8 @@ private RewriteOptions.Builder createBuilder(List inputPaths) throws IOExc return createBuilder(inputPaths, new ArrayList<>(), false); } - private RewriteOptions.Builder createBuilder(List inputPathsL, List inputPathsR, boolean joinColumnsOverwrite) - throws IOException { + private RewriteOptions.Builder createBuilder( + List inputPathsL, List inputPathsR, boolean joinColumnsOverwrite) throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); From 5b055c0ab74e2aeaae4dd858af3b6de20bfbfdb2 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Apr 2024 15:22:05 -0700 Subject: [PATCH 24/57] redesign file joiner in ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index e9868dd823..844ae9228d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -276,9 +276,9 @@ public void close() throws IOException { } public void processBlocks() throws IOException { - TransParquetFileReader readerR = inputFilesToJoin.peek(); - IndexCache indexCacheR = null; - int blockIdxR = -1; + TransParquetFileReader readerJoin = inputFilesToJoin.peek(); + IndexCache indexCacheJoin = null; + int blockIdxJoin = -1; while (!inputFiles.isEmpty()) { TransParquetFileReader reader = inputFiles.poll(); @@ -296,34 +296,35 @@ public void processBlocks() throws IOException { Map pathToChunk = blockMetaData.getColumns().stream().collect(Collectors.toMap(x -> x.getPath(), x -> x)); - if (readerR != null - && (blockIdxR == -1 - || ++blockIdxR - == readerR.getFooter().getBlocks().size())) { - blockIdxR = 0; - readerR = inputFilesToJoin.poll(); - Set columnPathsR = readerR.getFileMetaData().getSchema().getColumns().stream() + if (readerJoin != null + && (blockIdxJoin == -1 + || ++blockIdxJoin + == readerJoin.getFooter().getBlocks().size())) { + blockIdxJoin = 0; + readerJoin = inputFilesToJoin.poll(); + Set columnPathsJoin = readerJoin.getFileMetaData().getSchema().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); - if (indexCacheR != null) { - indexCacheR.clean(); + if (indexCacheJoin != null) { + indexCacheJoin.clean(); } - indexCacheR = IndexCache.create(readerR, columnPathsR, indexCacheStrategy, true); - indexCacheR.setBlockMetadata(readerR.getFooter().getBlocks().get(blockIdxR)); + indexCacheJoin = IndexCache.create(readerJoin, columnPathsJoin, indexCacheStrategy, true); + indexCacheJoin.setBlockMetadata( + readerJoin.getFooter().getBlocks().get(blockIdxJoin)); } else { - blockIdxR++; + blockIdxJoin++; } for (int outColumnIdx = 0; outColumnIdx < outSchema.getColumns().size(); outColumnIdx++) { ColumnPath colPath = ColumnPath.get( outSchema.getColumns().get(outColumnIdx).getPath()); - if (readerR != null) { - Optional chunkR = - readerR.getFooter().getBlocks().get(blockIdxR).getColumns().stream() + if (readerJoin != null) { + Optional chunkJoin = + readerJoin.getFooter().getBlocks().get(blockIdxJoin).getColumns().stream() .filter(x -> x.getPath().equals(colPath)) .findFirst(); - if (chunkR.isPresent() && (joinColumnsOverwrite || !columnPaths.contains(colPath))) { - processBlock(readerR, blockIdxR, outColumnIdx, indexCacheR, chunkR.get()); + if (chunkJoin.isPresent() && (joinColumnsOverwrite || !columnPaths.contains(colPath))) { + processBlock(readerJoin, blockIdxJoin, outColumnIdx, indexCacheJoin, chunkJoin.get()); } else { processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } @@ -342,8 +343,6 @@ public void processBlocks() throws IOException { } } - private void processBlocksFromReader(TransParquetFileReader reader) throws IOException {} - private void processBlock( TransParquetFileReader reader, int blockIdx, From b70f88f62b62d69bb15e03d2749b73cc44c9a450 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 28 Apr 2024 15:39:25 -0700 Subject: [PATCH 25/57] uncomment some code --- .../apache/parquet/hadoop/rewrite/ParquetRewriterTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index b070f481b4..4972000401 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -117,9 +117,9 @@ public class ParquetRewriterTest { @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { return new Object[][] { - // {"v1", "NONE", true}, - // {"v1", "PREFETCH_BLOCK", true}, - // {"v2", "PREFETCH_BLOCK", true}, + {"v1", "NONE", true}, + {"v1", "PREFETCH_BLOCK", true}, + {"v2", "PREFETCH_BLOCK", true}, {"v2", "PREFETCH_BLOCK", false} }; } From 270126b43c79dd2d495d4300e6f6443a156b57b4 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Fri, 3 May 2024 17:06:11 -0700 Subject: [PATCH 26/57] fix ParquetRewriter joiner test --- .../hadoop/rewrite/ParquetRewriterTest.java | 46 +++++++++---------- 1 file changed, 21 insertions(+), 25 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 4972000401..7dd6b2956d 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -763,7 +763,7 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) .collect(Collectors.toSet()); // TODO potentially too many checks, might need to be split into multiple tests - validateColumnData(Collections.emptySet(), Collections.emptySet(), null); // Verify data + validateColumnData(Collections.emptySet(), Collections.emptySet(), null, true); // Verify data assertEquals(expectSchema, actualSchema); // Verify schema validateCreatedBy(); // Verify original.created.by assertEquals(inputBloomFilters.keySet(), rBloomFilters); // Verify bloom filters @@ -839,38 +839,34 @@ private MessageType createSchemaR() { private void validateColumnData( Set prunePaths, Set nullifiedPaths, FileDecryptionProperties fileDecryptionProperties) throws IOException { + validateColumnData(prunePaths, nullifiedPaths, fileDecryptionProperties, false); + } + + private void validateColumnData( + Set prunePaths, + Set nullifiedPaths, + FileDecryptionProperties fileDecryptionProperties, + Boolean joinColumnsOverwrite) + throws IOException { ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), new Path(outputFile)) .withConf(conf) .withDecryption(fileDecryptionProperties) .build(); - List> fileContents = inputFiles.stream() - .map(x -> Arrays.stream(x.getFileContent()).collect(Collectors.toList())) + List filesMain = inputFiles.stream() + .flatMap(x -> Arrays.stream(x.getFileContent())) .collect(Collectors.toList()); - List> fileContentsJoined = inputFilesToJoinColumns.stream() - .map(x -> Arrays.stream(x.getFileContent()).collect(Collectors.toList())) + List filesJoined = inputFilesToJoinColumns.stream() + .flatMap(x -> Arrays.stream(x.getFileContent())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { - for (int i = 0; i < fileContents.size(); i++) { - if (rowIdx >= fileContents.get(i).size()) { - rowIdx -= fileContents.get(i).size(); - } else { - if (!fileContentsJoined.isEmpty()) { // todo check if joinColumnsOverwrite = true - SimpleGroup expGroup = fileContentsJoined.get(i).get(rowIdx); - GroupType fileSchema = expGroup.getType(); - if (fileSchema.containsField(name)) { - return expGroup; - } - } - SimpleGroup expGroup = fileContents.get(i).get(rowIdx); - GroupType fileSchema = expGroup.getType(); - if (fileSchema.containsField(name)) { - return expGroup; - } - } + if (joinColumnsOverwrite + && !filesJoined.isEmpty() + && filesJoined.get(0).getType().containsField(name)) { + return filesJoined.get(rowIdx); + } else { + return filesMain.get(rowIdx); } - throw new IllegalStateException( - "Group '" + name + "' at position " + rowIdx + " was not found in input files!"); }; int totalRows = @@ -1190,8 +1186,8 @@ private RewriteOptions.Builder createBuilder( .map(p -> HadoopInputFile.fromPathUnchecked(p, conf)) .collect(Collectors.toList()); builder = new RewriteOptions.Builder(parquetConf, inputsL, inputsR, outputPath); - builder.joinColumnsOverwrite(joinColumnsOverwrite); } + builder.joinColumnsOverwrite(joinColumnsOverwrite); return builder; } From 0dc1793017085072bfddc4ef16966df4bd7e3e19 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 24 Jul 2024 22:00:04 -0700 Subject: [PATCH 27/57] add initial ParquetJoiner implementation --- .../org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index fd3bcf5cd5..a21d1183b2 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -761,7 +761,7 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) .getSchema(); MessageType expectSchema = createSchema(); - Map> inputBloomFilters = allInputBloomFilters(null); + Map> inputBloomFilters = allInputBloomFilters(); Map> outputBloomFilters = allOutputBloomFilters(null); Set schemaRColumns = createSchemaR().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) From a53d1087347c7ee41fd53df273a760fa5fcf70b1 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 30 Jul 2024 22:59:15 -0700 Subject: [PATCH 28/57] add initial ParquetJoiner implementation --- .../hadoop/rewrite/ParquetRewriter.java | 42 ++++++---- .../hadoop/rewrite/RewriteOptions.java | 84 ++++++++++++------- 2 files changed, 79 insertions(+), 47 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 844ae9228d..a6b2514d8d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -113,38 +113,44 @@ public class ParquetRewriter implements Closeable { private MessageType outSchema; // The index cache strategy private final IndexCache.CacheStrategy indexCacheStrategy; - private final boolean joinColumnsOverwrite; + private final boolean overwriteInputWithJoinColumns; public ParquetRewriter(RewriteOptions options) throws IOException { this.newCodecName = options.getNewCodecName(); this.indexCacheStrategy = options.getIndexCacheStrategy(); - this.joinColumnsOverwrite = options.getJoinColumnsOverwrite(); + this.overwriteInputWithJoinColumns = options.getOverwriteInputWithJoinColumns(); ParquetConfiguration conf = options.getParquetConfiguration(); OutputFile out = options.getParquetOutputFile(); inputFiles.addAll(getFileReaders(options.getParquetInputFiles(), conf)); - inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoinColumns(), conf)); + inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoin(), conf)); ensureSameSchema(inputFiles); ensureSameSchema(inputFilesToJoin); LOG.info( - "Start rewriting {} input file(s) {} with {} groups of input file(s) to join {} to {}", - inputFilesToJoin.size(), - options.getParquetInputFilesToJoinColumns(), - inputFiles.size(), - options.getParquetInputFiles(), + "Start rewriting {} input file(s) {} to {}", + inputFiles.size() + inputFilesToJoin.size(), + Stream.concat(options.getParquetInputFiles().stream(), options.getParquetInputFilesToJoin().stream()) + .collect(Collectors.toList()), out); this.outSchema = getSchema(); this.outSchema = pruneColumnsInSchema(outSchema, options.getPruneColumns()); + + List allFiles; + if (options.getIgnoreJoinFilesMetadata()) { + allFiles = new ArrayList<>(inputFiles); + } else { + allFiles = Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) + .collect(Collectors.toList()); + } extraMetaData.put( ORIGINAL_CREATED_BY_KEY, - Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) + allFiles.stream() .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) .collect(Collectors.toSet()) .stream() .reduce((a, b) -> a + "\n" + b) .orElse("")); - Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) - .forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); + allFiles.forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); if (!inputFilesToJoin.isEmpty()) { List blocksRowCountsL = inputFiles.stream() @@ -200,7 +206,7 @@ private MessageType getSchema() { .getSchema() .getFields() .forEach(x -> { - if (!fieldNames.containsKey(x.getName()) || joinColumnsOverwrite) { + if (!fieldNames.containsKey(x.getName()) || overwriteInputWithJoinColumns) { fieldNames.put(x.getName(), x); } }); @@ -232,7 +238,7 @@ public ParquetRewriter( } this.inputFiles.add(reader); this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; - this.joinColumnsOverwrite = false; + this.overwriteInputWithJoinColumns = false; } private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { @@ -279,6 +285,7 @@ public void processBlocks() throws IOException { TransParquetFileReader readerJoin = inputFilesToJoin.peek(); IndexCache indexCacheJoin = null; int blockIdxJoin = -1; + List outColumns = outSchema.getColumns(); while (!inputFiles.isEmpty()) { TransParquetFileReader reader = inputFiles.poll(); @@ -315,15 +322,16 @@ public void processBlocks() throws IOException { blockIdxJoin++; } - for (int outColumnIdx = 0; outColumnIdx < outSchema.getColumns().size(); outColumnIdx++) { - ColumnPath colPath = ColumnPath.get( - outSchema.getColumns().get(outColumnIdx).getPath()); + for (int outColumnIdx = 0; outColumnIdx < outColumns.size(); outColumnIdx++) { + ColumnPath colPath = + ColumnPath.get(outColumns.get(outColumnIdx).getPath()); if (readerJoin != null) { Optional chunkJoin = readerJoin.getFooter().getBlocks().get(blockIdxJoin).getColumns().stream() .filter(x -> x.getPath().equals(colPath)) .findFirst(); - if (chunkJoin.isPresent() && (joinColumnsOverwrite || !columnPaths.contains(colPath))) { + if (chunkJoin.isPresent() + && (overwriteInputWithJoinColumns || !columnPaths.contains(colPath))) { processBlock(readerJoin, blockIdxJoin, outColumnIdx, indexCacheJoin, chunkJoin.get()); } else { processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 3b6fd73218..f2cbf7ec97 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -52,7 +52,7 @@ public class RewriteOptions { private final ParquetConfiguration conf; private final List inputFiles; - private final List inputFilesToJoinColumns; + private final List inputFilesToJoin; private final OutputFile outputFile; private final List pruneColumns; private final CompressionCodecName newCodecName; @@ -60,12 +60,13 @@ public class RewriteOptions { private final List encryptColumns; private final FileEncryptionProperties fileEncryptionProperties; private final IndexCache.CacheStrategy indexCacheStrategy; - private final boolean joinColumnsOverwrite; + private final boolean overwriteInputWithJoinColumns; + private final boolean ignoreJoinFilesMetadata; private RewriteOptions( ParquetConfiguration conf, List inputFiles, - List inputFilesToJoinColumns, + List inputFilesToJoin, OutputFile outputFile, List pruneColumns, CompressionCodecName newCodecName, @@ -73,10 +74,11 @@ private RewriteOptions( List encryptColumns, FileEncryptionProperties fileEncryptionProperties, IndexCache.CacheStrategy indexCacheStrategy, - boolean joinColumnsOverwrite) { + boolean overwriteInputWithJoinColumns, + boolean ignoreJoinFilesMetadata) { this.conf = conf; this.inputFiles = inputFiles; - this.inputFilesToJoinColumns = inputFilesToJoinColumns; + this.inputFilesToJoin = inputFilesToJoin; this.outputFile = outputFile; this.pruneColumns = pruneColumns; this.newCodecName = newCodecName; @@ -84,7 +86,8 @@ private RewriteOptions( this.encryptColumns = encryptColumns; this.fileEncryptionProperties = fileEncryptionProperties; this.indexCacheStrategy = indexCacheStrategy; - this.joinColumnsOverwrite = joinColumnsOverwrite; + this.overwriteInputWithJoinColumns = overwriteInputWithJoinColumns; + this.ignoreJoinFilesMetadata = ignoreJoinFilesMetadata; } /** @@ -133,13 +136,13 @@ public List getParquetInputFiles() { return inputFiles; } - /** TODO fix documentation after addition of InputFilesToJoinColumns + /** TODO fix documentation after addition of InputFilesToJoin * Gets the right {@link InputFile}s for the rewrite. * * @return a {@link List} of the associated right {@link InputFile}s */ - public List getParquetInputFilesToJoinColumns() { - return inputFilesToJoinColumns; + public List getParquetInputFilesToJoin() { + return inputFilesToJoin; } /** @@ -189,15 +192,19 @@ public IndexCache.CacheStrategy getIndexCacheStrategy() { return indexCacheStrategy; } - public boolean getJoinColumnsOverwrite() { - return joinColumnsOverwrite; + public boolean getOverwriteInputWithJoinColumns() { + return overwriteInputWithJoinColumns; + } + + public boolean getIgnoreJoinFilesMetadata() { + return ignoreJoinFilesMetadata; } // Builder to create a RewriterOptions. public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; - private final List inputFilesToJoinColumns; + private final List inputFilesToJoin; private final OutputFile outputFile; private List pruneColumns; private CompressionCodecName newCodecName; @@ -206,6 +213,7 @@ public static class Builder { private FileEncryptionProperties fileEncryptionProperties; private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; private boolean joinColumnsOverwrite = false; + private boolean ignoreJoinFilesMetadata = false; /** * Create a builder to create a RewriterOptions. @@ -253,7 +261,7 @@ public Builder(Configuration conf, List inputFiles, Path outputFile) { for (Path inputFile : inputFiles) { this.inputFiles.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); } - this.inputFilesToJoinColumns = new ArrayList<>(); + this.inputFilesToJoin = new ArrayList<>(); this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } @@ -275,19 +283,19 @@ public Builder(Configuration conf, List inputFiles, Path outputFile) { public Builder(ParquetConfiguration conf, List inputFiles, OutputFile outputFile) { this.conf = conf; this.inputFiles = inputFiles; - this.inputFilesToJoinColumns = new ArrayList<>(); + this.inputFilesToJoin = new ArrayList<>(); this.outputFile = outputFile; } - public Builder(Configuration conf, List inputFiles, List inputFilesToJoinColumns, Path outputFile) { + public Builder(Configuration conf, List inputFiles, List inputFilesToJoin, Path outputFile) { this.conf = new HadoopParquetConfiguration(conf); this.inputFiles = new ArrayList<>(inputFiles.size()); for (Path inputFile : inputFiles) { this.inputFiles.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); } - this.inputFilesToJoinColumns = new ArrayList<>(inputFilesToJoinColumns.size()); - for (Path inputFile : inputFilesToJoinColumns) { - this.inputFilesToJoinColumns.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); + this.inputFilesToJoin = new ArrayList<>(inputFilesToJoin.size()); + for (Path inputFile : inputFilesToJoin) { + this.inputFilesToJoin.add(HadoopInputFile.fromPathUnchecked(inputFile, conf)); } this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } @@ -295,11 +303,11 @@ public Builder(Configuration conf, List inputFiles, List inputFilesT public Builder( ParquetConfiguration conf, List inputFiles, - List inputFilesToJoinColumns, + List inputFilesToJoin, OutputFile outputFile) { this.conf = conf; this.inputFiles = inputFiles; - this.inputFilesToJoinColumns = inputFilesToJoinColumns; + this.inputFilesToJoin = inputFilesToJoin; this.outputFile = outputFile; } @@ -380,14 +388,14 @@ public Builder addInputFile(Path path) { return this; } - /** TODO fix documentation after addition of InputFilesToJoinColumns + /** TODO fix documentation after addition of InputFilesToJoin * Add an input file to read from. * * @param path input file path to read from * @return self */ public Builder addInputFileToJoinColumns(Path path) { - this.inputFilesToJoinColumns.add( + this.inputFilesToJoin.add( HadoopInputFile.fromPathUnchecked(path, ConfigurationUtil.createHadoopConfiguration(conf))); return this; } @@ -403,14 +411,14 @@ public Builder addInputFile(InputFile inputFile) { return this; } - /** TODO fix documentation after addition of InputFilesToJoinColumns - * Add an input file to read from. + /** + * Add a file to join to other input files. * - * @param inputFile input file to read from + * @param fileToJoin input file to join * @return self */ - public Builder addInputFilesToJoinColumns(InputFile inputFile) { - this.inputFilesToJoinColumns.add(inputFile); + public Builder addInputFilesToJoin(InputFile fileToJoin) { + this.inputFilesToJoin.add(fileToJoin); return this; } @@ -428,13 +436,28 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { return this; } - /** TODO fix documentation after addition of InputFilesToJoinColumns + /** + * Set a flag weather columns from join files need to overwrite columns from input files. + * + * @param joinColumnsOverwrite + * @return self */ public Builder joinColumnsOverwrite(boolean joinColumnsOverwrite) { this.joinColumnsOverwrite = joinColumnsOverwrite; return this; } + /** + * Set a flag weather metadata from join files should be ignored. + * + * @param ignoreJoinFilesMetadata + * @return self + */ + public Builder ignoreJoinFilesMetadata(boolean ignoreJoinFilesMetadata) { + this.ignoreJoinFilesMetadata = ignoreJoinFilesMetadata; + return this; + } + /** * Build the RewriterOptions. * @@ -478,7 +501,7 @@ public RewriteOptions build() { return new RewriteOptions( conf, inputFiles, - inputFilesToJoinColumns, + inputFilesToJoin, outputFile, pruneColumns, newCodecName, @@ -486,7 +509,8 @@ public RewriteOptions build() { encryptColumns, fileEncryptionProperties, indexCacheStrategy, - joinColumnsOverwrite); + joinColumnsOverwrite, + ignoreJoinFilesMetadata); } } } From 4da0b85bc3391921af8426c0cdcd955e3c64b865 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 5 Aug 2024 21:14:20 -0700 Subject: [PATCH 29/57] typo --- .../java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index f2cbf7ec97..d948197996 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -437,7 +437,7 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { } /** - * Set a flag weather columns from join files need to overwrite columns from input files. + * Set a flag whether columns from join files need to overwrite columns from input files. * * @param joinColumnsOverwrite * @return self From c5c7b3826c9954d9a91a9df298b91d23aac5e24d Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 5 Aug 2024 21:15:57 -0700 Subject: [PATCH 30/57] typo --- .../java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index d948197996..48947ceca3 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -448,7 +448,7 @@ public Builder joinColumnsOverwrite(boolean joinColumnsOverwrite) { } /** - * Set a flag weather metadata from join files should be ignored. + * Set a flag whether metadata from join files should be ignored. * * @param ignoreJoinFilesMetadata * @return self From 92c95dbba839b0d9f5684bd646391eac823e3e17 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 5 Aug 2024 21:23:12 -0700 Subject: [PATCH 31/57] typo --- .../apache/parquet/hadoop/rewrite/RewriteOptions.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 48947ceca3..27d4f8fad7 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -212,7 +212,7 @@ public static class Builder { private List encryptColumns; private FileEncryptionProperties fileEncryptionProperties; private IndexCache.CacheStrategy indexCacheStrategy = IndexCache.CacheStrategy.NONE; - private boolean joinColumnsOverwrite = false; + private boolean overwriteInputWithJoinColumns = false; private boolean ignoreJoinFilesMetadata = false; /** @@ -439,11 +439,11 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { /** * Set a flag whether columns from join files need to overwrite columns from input files. * - * @param joinColumnsOverwrite + * @param overwriteInputWithJoinColumns * @return self */ - public Builder joinColumnsOverwrite(boolean joinColumnsOverwrite) { - this.joinColumnsOverwrite = joinColumnsOverwrite; + public Builder overwriteInputWithJoinColumns(boolean overwriteInputWithJoinColumns) { + this.overwriteInputWithJoinColumns = overwriteInputWithJoinColumns; return this; } @@ -509,7 +509,7 @@ public RewriteOptions build() { encryptColumns, fileEncryptionProperties, indexCacheStrategy, - joinColumnsOverwrite, + overwriteInputWithJoinColumns, ignoreJoinFilesMetadata); } } From 86f7a4c4dbfaf239ecee8bc8c06aee6ee8473cea Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 5 Aug 2024 21:25:09 -0700 Subject: [PATCH 32/57] typo --- .../apache/parquet/hadoop/rewrite/ParquetRewriterTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index a21d1183b2..4826087125 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -1178,7 +1178,7 @@ private RewriteOptions.Builder createBuilder(List inputPaths) throws IOExc } private RewriteOptions.Builder createBuilder( - List inputPathsL, List inputPathsR, boolean joinColumnsOverwrite) throws IOException { + List inputPathsL, List inputPathsR, boolean overwriteInputWithJoinColumns) throws IOException { RewriteOptions.Builder builder; if (usingHadoop) { Path outputPath = new Path(outputFile); @@ -1193,7 +1193,7 @@ private RewriteOptions.Builder createBuilder( .collect(Collectors.toList()); builder = new RewriteOptions.Builder(parquetConf, inputsL, inputsR, outputPath); } - builder.joinColumnsOverwrite(joinColumnsOverwrite); + builder.overwriteInputWithJoinColumns(overwriteInputWithJoinColumns); return builder; } From 73a4af4bab983352602af8245dd57390db4fe625 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 5 Aug 2024 21:44:42 -0700 Subject: [PATCH 33/57] docs --- .../org/apache/parquet/hadoop/rewrite/RewriteOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 27d4f8fad7..e7c2ef4f8c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -467,8 +467,8 @@ public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); Preconditions.checkArgument( - inputFiles != null && !inputFiles.isEmpty(), - "Right side Input files can't be empty, if you don't need a join functionality then use other builders"); + inputFilesToJoin == null || !inputFiles.isEmpty(), + "Input files to join must be non-empty list or it can be left unset, it can't be an empty list"); if (pruneColumns != null) { if (maskColumns != null) { From 18feef42988f513cf5462ea347b5a5c4f56456b7 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 08:51:07 -0700 Subject: [PATCH 34/57] typo --- .../java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index e7c2ef4f8c..55f5408ffd 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -467,7 +467,7 @@ public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); Preconditions.checkArgument( - inputFilesToJoin == null || !inputFiles.isEmpty(), + inputFilesToJoin == null || !inputFilesToJoin.isEmpty(), "Input files to join must be non-empty list or it can be left unset, it can't be an empty list"); if (pruneColumns != null) { From b24bffae3b5b434a12eb97d6b4ba4f51d5eefd4f Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 09:40:44 -0700 Subject: [PATCH 35/57] add getExtraMetadata() --- .../hadoop/rewrite/ParquetRewriter.java | 43 +++++++++++-------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index a6b2514d8d..0da869ad77 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -102,7 +102,7 @@ public class ParquetRewriter implements Closeable { private Map maskColumns = null; private Set encryptColumns = null; private boolean encryptMode = false; - private final Map extraMetaData = new HashMap<>(); + private final Map extraMetaData; // Writer to rewrite the input files private final ParquetFileWriter writer; // Number of blocks written which is used to keep track of the actual row group ordinal @@ -134,23 +134,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { this.outSchema = getSchema(); this.outSchema = pruneColumnsInSchema(outSchema, options.getPruneColumns()); - - List allFiles; - if (options.getIgnoreJoinFilesMetadata()) { - allFiles = new ArrayList<>(inputFiles); - } else { - allFiles = Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) - .collect(Collectors.toList()); - } - extraMetaData.put( - ORIGINAL_CREATED_BY_KEY, - allFiles.stream() - .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) - .collect(Collectors.toSet()) - .stream() - .reduce((a, b) -> a + "\n" + b) - .orElse("")); - allFiles.forEach(x -> extraMetaData.putAll(x.getFileMetaData().getKeyValueMetaData())); + this.extraMetaData = getExtraMetadata(options); if (!inputFilesToJoin.isEmpty()) { List blocksRowCountsL = inputFiles.stream() @@ -214,6 +198,27 @@ private MessageType getSchema() { } } + private Map getExtraMetadata(RewriteOptions options) { + List allFiles; + if (options.getIgnoreJoinFilesMetadata()) { + allFiles = new ArrayList<>(inputFiles); + } else { + allFiles = Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()) + .collect(Collectors.toList()); + } + Map result = new HashMap<>(); + result.put( + ORIGINAL_CREATED_BY_KEY, + allFiles.stream() + .map(x -> x.getFooter().getFileMetaData().getCreatedBy()) + .collect(Collectors.toSet()) + .stream() + .reduce((a, b) -> a + "\n" + b) + .orElse("")); + allFiles.forEach(x -> result.putAll(x.getFileMetaData().getKeyValueMetaData())); + return result; + } + // Ctor for legacy CompressionConverter and ColumnMasker public ParquetRewriter( TransParquetFileReader reader, @@ -228,7 +233,7 @@ public ParquetRewriter( this.outSchema = outSchema; this.newCodecName = codecName; originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; - extraMetaData.putAll(meta.getFileMetaData().getKeyValueMetaData()); + extraMetaData = meta.getFileMetaData().getKeyValueMetaData(); extraMetaData.put(ORIGINAL_CREATED_BY_KEY, originalCreatedBy); if (maskColumns != null && maskMode != null) { this.maskColumns = new HashMap<>(); From 21a5926eeead565d77f89099b787b194922a3c4a Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 09:46:59 -0700 Subject: [PATCH 36/57] extract ensureRowCount() --- .../hadoop/rewrite/ParquetRewriter.java | 79 ++++++++++--------- 1 file changed, 42 insertions(+), 37 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 0da869ad77..20365dadd0 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -125,6 +125,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { inputFilesToJoin.addAll(getFileReaders(options.getParquetInputFilesToJoin(), conf)); ensureSameSchema(inputFiles); ensureSameSchema(inputFilesToJoin); + ensureRowCount(); LOG.info( "Start rewriting {} input file(s) {} to {}", inputFiles.size() + inputFilesToJoin.size(), @@ -136,20 +137,6 @@ public ParquetRewriter(RewriteOptions options) throws IOException { this.outSchema = pruneColumnsInSchema(outSchema, options.getPruneColumns()); this.extraMetaData = getExtraMetadata(options); - if (!inputFilesToJoin.isEmpty()) { - List blocksRowCountsL = inputFiles.stream() - .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) - .collect(Collectors.toList()); - List blocksRowCountsR = inputFilesToJoin.stream() - .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) - .collect(Collectors.toList()); - if (!blocksRowCountsL.equals(blocksRowCountsR)) { - throw new IllegalArgumentException( - "The number of rows in each block must match! Left blocks row counts: " + blocksRowCountsL - + ", right blocks row counts" + blocksRowCountsR + "."); - } - } - if (options.getMaskColumns() != null) { this.maskColumns = new HashMap<>(); for (Map.Entry col : options.getMaskColumns().entrySet()) { @@ -176,6 +163,35 @@ public ParquetRewriter(RewriteOptions options) throws IOException { writer.start(); } + + // Ctor for legacy CompressionConverter and ColumnMasker + public ParquetRewriter( + TransParquetFileReader reader, + ParquetFileWriter writer, + ParquetMetadata meta, + MessageType outSchema, + String originalCreatedBy, + CompressionCodecName codecName, + List maskColumns, + MaskMode maskMode) { + this.writer = writer; + this.outSchema = outSchema; + this.newCodecName = codecName; + originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; + extraMetaData = meta.getFileMetaData().getKeyValueMetaData(); + extraMetaData.put(ORIGINAL_CREATED_BY_KEY, originalCreatedBy); + if (maskColumns != null && maskMode != null) { + this.maskColumns = new HashMap<>(); + for (String col : maskColumns) { + this.maskColumns.put(ColumnPath.fromDotString(col), maskMode); + } + } + this.inputFiles.add(reader); + this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; + this.overwriteInputWithJoinColumns = false; + } + + private MessageType getSchema() { MessageType schemaMain = inputFiles.peek().getFooter().getFileMetaData().getSchema(); if (inputFilesToJoin.isEmpty()) { @@ -219,31 +235,20 @@ private Map getExtraMetadata(RewriteOptions options) { return result; } - // Ctor for legacy CompressionConverter and ColumnMasker - public ParquetRewriter( - TransParquetFileReader reader, - ParquetFileWriter writer, - ParquetMetadata meta, - MessageType outSchema, - String originalCreatedBy, - CompressionCodecName codecName, - List maskColumns, - MaskMode maskMode) { - this.writer = writer; - this.outSchema = outSchema; - this.newCodecName = codecName; - originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; - extraMetaData = meta.getFileMetaData().getKeyValueMetaData(); - extraMetaData.put(ORIGINAL_CREATED_BY_KEY, originalCreatedBy); - if (maskColumns != null && maskMode != null) { - this.maskColumns = new HashMap<>(); - for (String col : maskColumns) { - this.maskColumns.put(ColumnPath.fromDotString(col), maskMode); + private void ensureRowCount() { + if (!inputFilesToJoin.isEmpty()) { + List blocksRowCountsL = inputFiles.stream() + .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) + .collect(Collectors.toList()); + List blocksRowCountsR = inputFilesToJoin.stream() + .flatMap(x -> x.getFooter().getBlocks().stream().map(BlockMetaData::getRowCount)) + .collect(Collectors.toList()); + if (!blocksRowCountsL.equals(blocksRowCountsR)) { + throw new IllegalArgumentException( + "The number of rows in each block must match! Left blocks row counts: " + blocksRowCountsL + + ", right blocks row counts" + blocksRowCountsR + "."); } } - this.inputFiles.add(reader); - this.indexCacheStrategy = IndexCache.CacheStrategy.NONE; - this.overwriteInputWithJoinColumns = false; } private Queue getFileReaders(List inputFiles, ParquetConfiguration conf) { From c521a95b2cd6654c1863d045c5bee3c0c6acc8dd Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 09:48:21 -0700 Subject: [PATCH 37/57] typo --- .../org/apache/parquet/hadoop/rewrite/ParquetRewriter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 20365dadd0..4c942ab2b1 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -336,13 +336,13 @@ public void processBlocks() throws IOException { ColumnPath colPath = ColumnPath.get(outColumns.get(outColumnIdx).getPath()); if (readerJoin != null) { - Optional chunkJoin = + Optional chunkToJoin = readerJoin.getFooter().getBlocks().get(blockIdxJoin).getColumns().stream() .filter(x -> x.getPath().equals(colPath)) .findFirst(); - if (chunkJoin.isPresent() + if (chunkToJoin.isPresent() && (overwriteInputWithJoinColumns || !columnPaths.contains(colPath))) { - processBlock(readerJoin, blockIdxJoin, outColumnIdx, indexCacheJoin, chunkJoin.get()); + processBlock(readerJoin, blockIdxJoin, outColumnIdx, indexCacheJoin, chunkToJoin.get()); } else { processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } From 1ea675595d1c23aceab1d29b91a012d3b1c71698 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 09:53:33 -0700 Subject: [PATCH 38/57] typo --- .../hadoop/rewrite/ParquetRewriter.java | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 4c942ab2b1..638df243ed 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -292,9 +292,9 @@ public void close() throws IOException { } public void processBlocks() throws IOException { - TransParquetFileReader readerJoin = inputFilesToJoin.peek(); - IndexCache indexCacheJoin = null; - int blockIdxJoin = -1; + TransParquetFileReader readerToJoin = inputFilesToJoin.peek(); + IndexCache indexCacheToJoin = null; + int blockIdxToJoin = -1; List outColumns = outSchema.getColumns(); while (!inputFiles.isEmpty()) { @@ -313,36 +313,36 @@ public void processBlocks() throws IOException { Map pathToChunk = blockMetaData.getColumns().stream().collect(Collectors.toMap(x -> x.getPath(), x -> x)); - if (readerJoin != null - && (blockIdxJoin == -1 - || ++blockIdxJoin - == readerJoin.getFooter().getBlocks().size())) { - blockIdxJoin = 0; - readerJoin = inputFilesToJoin.poll(); - Set columnPathsJoin = readerJoin.getFileMetaData().getSchema().getColumns().stream() + if (readerToJoin != null + && (blockIdxToJoin == -1 + || ++blockIdxToJoin + == readerToJoin.getFooter().getBlocks().size())) { + blockIdxToJoin = 0; + readerToJoin = inputFilesToJoin.poll(); + Set columnPathsToJoin = readerToJoin.getFileMetaData().getSchema().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); - if (indexCacheJoin != null) { - indexCacheJoin.clean(); + if (indexCacheToJoin != null) { + indexCacheToJoin.clean(); } - indexCacheJoin = IndexCache.create(readerJoin, columnPathsJoin, indexCacheStrategy, true); - indexCacheJoin.setBlockMetadata( - readerJoin.getFooter().getBlocks().get(blockIdxJoin)); + indexCacheToJoin = IndexCache.create(readerToJoin, columnPathsToJoin, indexCacheStrategy, true); + indexCacheToJoin.setBlockMetadata( + readerToJoin.getFooter().getBlocks().get(blockIdxToJoin)); } else { - blockIdxJoin++; + blockIdxToJoin++; } for (int outColumnIdx = 0; outColumnIdx < outColumns.size(); outColumnIdx++) { ColumnPath colPath = ColumnPath.get(outColumns.get(outColumnIdx).getPath()); - if (readerJoin != null) { + if (readerToJoin != null) { Optional chunkToJoin = - readerJoin.getFooter().getBlocks().get(blockIdxJoin).getColumns().stream() + readerToJoin.getFooter().getBlocks().get(blockIdxToJoin).getColumns().stream() .filter(x -> x.getPath().equals(colPath)) .findFirst(); if (chunkToJoin.isPresent() && (overwriteInputWithJoinColumns || !columnPaths.contains(colPath))) { - processBlock(readerJoin, blockIdxJoin, outColumnIdx, indexCacheJoin, chunkToJoin.get()); + processBlock(readerToJoin, blockIdxToJoin, outColumnIdx, indexCacheToJoin, chunkToJoin.get()); } else { processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } From f2e01a2e0320344fcc8e1319f03e1d66d5338fca Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 10:04:35 -0700 Subject: [PATCH 39/57] add logging into getSchema() --- .../parquet/hadoop/rewrite/ParquetRewriter.java | 11 +++++++---- .../parquet/hadoop/rewrite/ParquetRewriterTest.java | 6 +++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 638df243ed..e40f3a1674 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -163,7 +163,6 @@ public ParquetRewriter(RewriteOptions options) throws IOException { writer.start(); } - // Ctor for legacy CompressionConverter and ColumnMasker public ParquetRewriter( TransParquetFileReader reader, @@ -191,7 +190,6 @@ public ParquetRewriter( this.overwriteInputWithJoinColumns = false; } - private MessageType getSchema() { MessageType schemaMain = inputFiles.peek().getFooter().getFileMetaData().getSchema(); if (inputFilesToJoin.isEmpty()) { @@ -206,7 +204,11 @@ private MessageType getSchema() { .getSchema() .getFields() .forEach(x -> { - if (!fieldNames.containsKey(x.getName()) || overwriteInputWithJoinColumns) { + if (!fieldNames.containsKey(x.getName())) { + LOG.info("Column {} is added to the output from inputFilesToJoin side", x.getName()); + fieldNames.put(x.getName(), x); + } else if (overwriteInputWithJoinColumns) { + LOG.info("Column {} in inputFiles is overwritten by inputFilesToJoin side", x.getName()); fieldNames.put(x.getName(), x); } }); @@ -342,7 +344,8 @@ public void processBlocks() throws IOException { .findFirst(); if (chunkToJoin.isPresent() && (overwriteInputWithJoinColumns || !columnPaths.contains(colPath))) { - processBlock(readerToJoin, blockIdxToJoin, outColumnIdx, indexCacheToJoin, chunkToJoin.get()); + processBlock( + readerToJoin, blockIdxToJoin, outColumnIdx, indexCacheToJoin, chunkToJoin.get()); } else { processBlock(reader, blockIdx, outColumnIdx, indexCache, pathToChunk.get(colPath)); } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 4826087125..2ba8e4480a 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -120,9 +120,9 @@ public class ParquetRewriterTest { @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { return new Object[][] { - {"v1", "NONE", true}, - {"v1", "PREFETCH_BLOCK", true}, - {"v2", "PREFETCH_BLOCK", true}, + // {"v1", "NONE", true}, + // {"v1", "PREFETCH_BLOCK", true}, + // {"v2", "PREFETCH_BLOCK", true}, {"v2", "PREFETCH_BLOCK", false} }; } From d393125452baf0db5ec140360ad815bb6f7636c5 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 10:05:01 -0700 Subject: [PATCH 40/57] typo --- .../apache/parquet/hadoop/rewrite/ParquetRewriterTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 2ba8e4480a..4826087125 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -120,9 +120,9 @@ public class ParquetRewriterTest { @Parameterized.Parameters(name = "WriterVersion = {0}, IndexCacheStrategy = {1}, UsingHadoop = {2}") public static Object[][] parameters() { return new Object[][] { - // {"v1", "NONE", true}, - // {"v1", "PREFETCH_BLOCK", true}, - // {"v2", "PREFETCH_BLOCK", true}, + {"v1", "NONE", true}, + {"v1", "PREFETCH_BLOCK", true}, + {"v2", "PREFETCH_BLOCK", true}, {"v2", "PREFETCH_BLOCK", false} }; } From 64d3bb2a3283aedd11ba9fb3b7d12864fe0b3d28 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 10:27:41 -0700 Subject: [PATCH 41/57] add closing of input files readers --- .../hadoop/rewrite/ParquetRewriter.java | 41 +++++++++++-------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index e40f3a1674..4d27b705d1 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -294,9 +294,9 @@ public void close() throws IOException { } public void processBlocks() throws IOException { - TransParquetFileReader readerToJoin = inputFilesToJoin.peek(); + TransParquetFileReader readerToJoin = null; IndexCache indexCacheToJoin = null; - int blockIdxToJoin = -1; + int blockIdxToJoin = 0; List outColumns = outSchema.getColumns(); while (!inputFiles.isEmpty()) { @@ -315,23 +315,26 @@ public void processBlocks() throws IOException { Map pathToChunk = blockMetaData.getColumns().stream().collect(Collectors.toMap(x -> x.getPath(), x -> x)); - if (readerToJoin != null - && (blockIdxToJoin == -1 - || ++blockIdxToJoin - == readerToJoin.getFooter().getBlocks().size())) { - blockIdxToJoin = 0; - readerToJoin = inputFilesToJoin.poll(); - Set columnPathsToJoin = readerToJoin.getFileMetaData().getSchema().getColumns().stream() - .map(x -> ColumnPath.get(x.getPath())) - .collect(Collectors.toSet()); - if (indexCacheToJoin != null) { - indexCacheToJoin.clean(); + if (!inputFilesToJoin.isEmpty()) { + if (readerToJoin == null + || ++blockIdxToJoin + == readerToJoin.getFooter().getBlocks().size()) { + if (readerToJoin != null) readerToJoin.close(); + blockIdxToJoin = 0; + readerToJoin = inputFilesToJoin.poll(); + Set columnPathsToJoin = + readerToJoin.getFileMetaData().getSchema().getColumns().stream() + .map(x -> ColumnPath.get(x.getPath())) + .collect(Collectors.toSet()); + if (indexCacheToJoin != null) { + indexCacheToJoin.clean(); + } + indexCacheToJoin = IndexCache.create(readerToJoin, columnPathsToJoin, indexCacheStrategy, true); + indexCacheToJoin.setBlockMetadata( + readerToJoin.getFooter().getBlocks().get(blockIdxToJoin)); + } else { + blockIdxToJoin++; } - indexCacheToJoin = IndexCache.create(readerToJoin, columnPathsToJoin, indexCacheStrategy, true); - indexCacheToJoin.setBlockMetadata( - readerToJoin.getFooter().getBlocks().get(blockIdxToJoin)); - } else { - blockIdxToJoin++; } for (int outColumnIdx = 0; outColumnIdx < outColumns.size(); outColumnIdx++) { @@ -361,7 +364,9 @@ public void processBlocks() throws IOException { indexCache.clean(); LOG.info("Finish rewriting input file: {}", reader.getFile()); + reader.close(); } + if (readerToJoin != null) readerToJoin.close(); } private void processBlock( From f50666a19a037d2107671ab4bbbf99b8b9345b67 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 11:17:14 -0700 Subject: [PATCH 42/57] fix RewriteOptions builder for inputFilesToJoin --- .../org/apache/parquet/hadoop/rewrite/RewriteOptions.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 55f5408ffd..82ef2f8086 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -466,9 +466,6 @@ public Builder ignoreJoinFilesMetadata(boolean ignoreJoinFilesMetadata) { public RewriteOptions build() { Preconditions.checkArgument(inputFiles != null && !inputFiles.isEmpty(), "Input file is required"); Preconditions.checkArgument(outputFile != null, "Output file is required"); - Preconditions.checkArgument( - inputFilesToJoin == null || !inputFilesToJoin.isEmpty(), - "Input files to join must be non-empty list or it can be left unset, it can't be an empty list"); if (pruneColumns != null) { if (maskColumns != null) { @@ -501,7 +498,7 @@ public RewriteOptions build() { return new RewriteOptions( conf, inputFiles, - inputFilesToJoin, + (inputFilesToJoin != null ? inputFilesToJoin : new ArrayList<>()), outputFile, pruneColumns, newCodecName, From ae9589d4f9a88da53ecab26d0d9eb8051ae1fb08 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 7 Aug 2024 12:03:17 -0700 Subject: [PATCH 43/57] fix ParquetRewriter constructor --- .../apache/parquet/hadoop/rewrite/ParquetRewriter.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 4d27b705d1..7ced56d566 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -176,9 +176,12 @@ public ParquetRewriter( this.writer = writer; this.outSchema = outSchema; this.newCodecName = codecName; - originalCreatedBy = originalCreatedBy == null ? meta.getFileMetaData().getCreatedBy() : originalCreatedBy; - extraMetaData = meta.getFileMetaData().getKeyValueMetaData(); - extraMetaData.put(ORIGINAL_CREATED_BY_KEY, originalCreatedBy); + extraMetaData = new HashMap<>(meta.getFileMetaData().getKeyValueMetaData()); + extraMetaData.put( + ORIGINAL_CREATED_BY_KEY, + originalCreatedBy != null + ? originalCreatedBy + : meta.getFileMetaData().getCreatedBy()); if (maskColumns != null && maskMode != null) { this.maskColumns = new HashMap<>(); for (String col : maskColumns) { From 9157960cc728739787dfd2a12672bfc70702e986 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 13 Aug 2024 21:39:20 -0700 Subject: [PATCH 44/57] extend tests for ParquetRewriter --- .../hadoop/rewrite/ParquetRewriterTest.java | 35 +++++++++++++------ 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 4826087125..c73f923cd9 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -739,10 +739,7 @@ private void testSingleInputFileSetup(String... bloomFilterEnabledColumns) throw @Test public void testStitchTwoInputs() throws Exception { - // todo add a case when a number of main input files and joined input files do not match - // for that we need to have a capability of specifying the number of rows in the output file - // right now a withRowGroupSize doesn't allow to strictly set a number of rows in a group - testTwoInputFileGroups(); + testOneInputFileManyInputFilesToJoin(); List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); @@ -795,17 +792,35 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) }); } - private void testTwoInputFileGroups() throws IOException { + private void testOneInputFileManyInputFilesToJoin() throws IOException { inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaL()) + .withNumRecord(numRecord) + .withRowGroupSize(1 * 1024 * 1024) .withCodec("GZIP") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - inputFilesToJoinColumns = Lists.newArrayList(new TestFileBuilder(conf, createSchemaR()) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build()); + + List rowGroupRowCounts = ParquetFileReader.readFooter( + conf, + new Path(inputFiles.get(0).getFileName()), + ParquetMetadataConverter.NO_FILTER + ) + .getBlocks() + .stream() + .map(BlockMetaData::getRowCount) + .collect(Collectors.toList()); + + for (long count: rowGroupRowCounts) { + inputFilesToJoinColumns.add( + new TestFileBuilder(conf, createSchemaR()) + .withNumRecord((int)count) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build() + ); + } } private MessageType createSchema() { From 3b722e4e21c0eb49ea8fa7a1fe704bb9fb55b335 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Tue, 13 Aug 2024 21:53:36 -0700 Subject: [PATCH 45/57] spotless --- .../hadoop/rewrite/ParquetRewriterTest.java | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index c73f923cd9..e52abd86d3 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -802,24 +802,19 @@ private void testOneInputFileManyInputFilesToJoin() throws IOException { .build()); List rowGroupRowCounts = ParquetFileReader.readFooter( - conf, - new Path(inputFiles.get(0).getFileName()), - ParquetMetadataConverter.NO_FILTER - ) + conf, new Path(inputFiles.get(0).getFileName()), ParquetMetadataConverter.NO_FILTER) .getBlocks() .stream() .map(BlockMetaData::getRowCount) .collect(Collectors.toList()); - for (long count: rowGroupRowCounts) { - inputFilesToJoinColumns.add( - new TestFileBuilder(conf, createSchemaR()) - .withNumRecord((int)count) - .withCodec("UNCOMPRESSED") - .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) - .withWriterVersion(writerVersion) - .build() - ); + for (long count : rowGroupRowCounts) { + inputFilesToJoinColumns.add(new TestFileBuilder(conf, createSchemaR()) + .withNumRecord((int) count) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); } } From bdba14ca97cc9be2b3003bae982d8c90df51d78f Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 14 Aug 2024 21:55:26 -0700 Subject: [PATCH 46/57] refactor ParquetRewriterTest --- .../hadoop/rewrite/ParquetRewriterTest.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index e52abd86d3..f0d93d0cc0 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -177,7 +177,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E null); // Verify the data are not changed for the columns not pruned - validateColumnData(new HashSet<>(pruneColumns), Collections.emptySet(), null); + validateColumnData(new HashSet<>(pruneColumns), Collections.emptySet(), null, false); // Verify the page index validatePageIndex(new HashMap() { @@ -255,7 +255,7 @@ private void testPruneNullifyTranslateCodec(List inputPaths) throws Except null); // Verify the data are not changed for the columns not pruned - validateColumnData(new HashSet<>(pruneColumns), maskColumns.keySet(), null); + validateColumnData(new HashSet<>(pruneColumns), maskColumns.keySet(), null, false); // Verify the page index validatePageIndex(new HashMap() { @@ -336,7 +336,7 @@ private void testPruneEncryptTranslateCodec(List inputPaths) throws Except fileDecryptionProperties); // Verify the data are not changed for the columns not pruned - validateColumnData(new HashSet<>(pruneColumns), Collections.emptySet(), fileDecryptionProperties); + validateColumnData(new HashSet<>(pruneColumns), Collections.emptySet(), fileDecryptionProperties, false); // Verify column encryption ParquetMetadata metaData = getFileMetaData(outputFile, fileDecryptionProperties); @@ -497,7 +497,7 @@ private void testNullifyAndEncryptColumn(List inputPaths) throws Exception // Verify the data are not changed for non-encrypted and non-masked columns. // Also make sure the masked column is nullified. - validateColumnData(Collections.emptySet(), maskColumns.keySet(), fileDecryptionProperties); + validateColumnData(Collections.emptySet(), maskColumns.keySet(), fileDecryptionProperties, false); // Verify the page index validatePageIndex(new HashMap() { @@ -587,7 +587,7 @@ public void testMergeTwoFilesOnly() throws Exception { null); // Verify the merged data are not changed - validateColumnData(Collections.emptySet(), Collections.emptySet(), null); + validateColumnData(Collections.emptySet(), Collections.emptySet(), null, false); // Verify the page index validatePageIndex(new HashMap() { @@ -738,8 +738,8 @@ private void testSingleInputFileSetup(String... bloomFilterEnabledColumns) throw } @Test - public void testStitchTwoInputs() throws Exception { - testOneInputFileManyInputFilesToJoin(); + public void testOneInputFileManyInputFilesToJoin() throws Exception { + testOneInputFileManyInputFilesToJoinSetup(); List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); @@ -792,7 +792,7 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) }); } - private void testOneInputFileManyInputFilesToJoin() throws IOException { + private void testOneInputFileManyInputFilesToJoinSetup() throws IOException { inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaL()) .withNumRecord(numRecord) .withRowGroupSize(1 * 1024 * 1024) @@ -854,12 +854,6 @@ private MessageType createSchemaR() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } - private void validateColumnData( - Set prunePaths, Set nullifiedPaths, FileDecryptionProperties fileDecryptionProperties) - throws IOException { - validateColumnData(prunePaths, nullifiedPaths, fileDecryptionProperties, false); - } - private void validateColumnData( Set prunePaths, Set nullifiedPaths, From a89eba6534529e642b5ae2d773a53a3452d8f151 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 15 Aug 2024 21:56:26 -0700 Subject: [PATCH 47/57] add tests into ParquetRewriterTest --- .../hadoop/rewrite/ParquetRewriterTest.java | 56 +++++++++++++------ 1 file changed, 39 insertions(+), 17 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index f0d93d0cc0..d7e57b7c87 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -110,7 +110,7 @@ public class ParquetRewriterTest { private final boolean usingHadoop; private List inputFiles = Lists.newArrayList(); - private List inputFilesToJoinColumns = Lists.newArrayList(); + private List inputFilesToJoin = Lists.newArrayList(); private String outputFile = null; private ParquetRewriter rewriter = null; @@ -197,7 +197,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E @Before public void setUp() { outputFile = TestFileBuilder.createTempFile("test"); - inputFilesToJoinColumns = new ArrayList<>(); + inputFilesToJoin = new ArrayList<>(); } @Test @@ -607,6 +607,15 @@ public void testMergeTwoFilesOnly() throws Exception { @Test(expected = InvalidSchemaException.class) public void testMergeTwoFilesWithDifferentSchema() throws Exception { + testMergeTwoFilesWithDifferentSchemaSetup(true); + } + + @Test(expected = InvalidSchemaException.class) + public void testMergeTwoFilesToJoinWithDifferentSchema() throws Exception { + testMergeTwoFilesWithDifferentSchemaSetup(false); + } + + public void testMergeTwoFilesWithDifferentSchemaSetup(boolean wrongSchemaInInputFile) throws Exception { MessageType schema1 = new MessageType( "schema", new PrimitiveType(OPTIONAL, INT64, "DocId"), @@ -629,18 +638,32 @@ public void testMergeTwoFilesWithDifferentSchema() throws Exception { .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - inputFiles.add(new TestFileBuilder(conf, schema2) + inputFilesToJoin.add(new TestFileBuilder(conf, schema1) .withNumRecord(numRecord) .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) .withWriterVersion(writerVersion) .build()); - - List inputPaths = new ArrayList<>(); - for (EncryptionTestFile inputFile : inputFiles) { - inputPaths.add(new Path(inputFile.getFileName())); + if (wrongSchemaInInputFile) { + inputFiles.add(new TestFileBuilder(conf, schema2) + .withNumRecord(numRecord) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); + } else { + inputFilesToJoin.add(new TestFileBuilder(conf, schema2) + .withNumRecord(numRecord) + .withCodec("UNCOMPRESSED") + .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) + .withWriterVersion(writerVersion) + .build()); } - RewriteOptions.Builder builder = createBuilder(inputPaths); + + RewriteOptions.Builder builder = createBuilder( + inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()), + inputFilesToJoin.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()), + false); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); // This should throw an exception because the schemas are different @@ -743,9 +766,8 @@ public void testOneInputFileManyInputFilesToJoin() throws Exception { List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); - List inputPathsR = inputFilesToJoinColumns.stream() - .map(y -> new Path(y.getFileName())) - .collect(Collectors.toList()); + List inputPathsR = + inputFilesToJoin.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList()); RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR, true); RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); @@ -809,7 +831,7 @@ conf, new Path(inputFiles.get(0).getFileName()), ParquetMetadataConverter.NO_FIL .collect(Collectors.toList()); for (long count : rowGroupRowCounts) { - inputFilesToJoinColumns.add(new TestFileBuilder(conf, createSchemaR()) + inputFilesToJoin.add(new TestFileBuilder(conf, createSchemaR()) .withNumRecord((int) count) .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) @@ -868,7 +890,7 @@ private void validateColumnData( List filesMain = inputFiles.stream() .flatMap(x -> Arrays.stream(x.getFileContent())) .collect(Collectors.toList()); - List filesJoined = inputFilesToJoinColumns.stream() + List filesJoined = inputFilesToJoin.stream() .flatMap(x -> Arrays.stream(x.getFileContent())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { @@ -1089,8 +1111,8 @@ private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData private void validateCreatedBy() throws Exception { Set createdBySet = new HashSet<>(); - List inFiles = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) - .collect(Collectors.toList()); + List inFiles = + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()).collect(Collectors.toList()); for (EncryptionTestFile inputFile : inFiles) { ParquetMetadata pmd = getFileMetaData(inputFile.getFileName(), null); createdBySet.add(pmd.getFileMetaData().getCreatedBy()); @@ -1133,8 +1155,8 @@ private void validateRowGroupRowCount() throws Exception { private Map> allInputBloomFilters() throws Exception { Map> inputBloomFilters = new HashMap<>(); - List files = Stream.concat(inputFiles.stream(), inputFilesToJoinColumns.stream()) - .collect(Collectors.toList()); + List files = + Stream.concat(inputFiles.stream(), inputFilesToJoin.stream()).collect(Collectors.toList()); for (EncryptionTestFile inputFile : files) { Map> bloomFilters = allBloomFilters(inputFile.getFileName(), null); for (Map.Entry> entry : bloomFilters.entrySet()) { From f674bcf5e41ad71d97f8aa0a74d9ddb5f1646b3c Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 25 Aug 2024 22:13:04 -0700 Subject: [PATCH 48/57] extend tests in ParquetRewriterTest for joiner part --- .../hadoop/rewrite/ParquetRewriter.java | 2 + .../hadoop/rewrite/ParquetRewriterTest.java | 288 +++++++++--------- 2 files changed, 140 insertions(+), 150 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 7ced56d566..1258568941 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -337,6 +337,8 @@ public void processBlocks() throws IOException { readerToJoin.getFooter().getBlocks().get(blockIdxToJoin)); } else { blockIdxToJoin++; + indexCacheToJoin.setBlockMetadata( + readerToJoin.getFooter().getBlocks().get(blockIdxToJoin)); } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index d7e57b7c87..b209f44915 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -33,18 +33,15 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; @@ -164,7 +161,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E rewriter.close(); // Verify the schema is not changed for the columns not pruned - validateSchema(); + validateSchemaWithGenderColumnPruned(); // Verify codec has been translated verifyCodec( @@ -180,14 +177,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E validateColumnData(new HashSet<>(pruneColumns), Collections.emptySet(), null, false); // Verify the page index - validatePageIndex(new HashMap() { - { - put(0, 0); - put(1, 1); - put(2, 3); - put(3, 4); - } - }); + validatePageIndex(new HashSet<>(), false); // Verify original.created.by is preserved validateCreatedBy(); @@ -242,7 +232,7 @@ private void testPruneNullifyTranslateCodec(List inputPaths) throws Except rewriter.close(); // Verify the schema are not changed for the columns not pruned - validateSchema(); + validateSchemaWithGenderColumnPruned(); // Verify codec has been translated verifyCodec( @@ -258,13 +248,7 @@ private void testPruneNullifyTranslateCodec(List inputPaths) throws Except validateColumnData(new HashSet<>(pruneColumns), maskColumns.keySet(), null, false); // Verify the page index - validatePageIndex(new HashMap() { - { - put(0, 0); - put(1, 1); - put(2, 3); - } - }); + validatePageIndex(ImmutableSet.of("Links.Forward"), false); // Verify original.created.by is preserved validateCreatedBy(); @@ -322,7 +306,7 @@ private void testPruneEncryptTranslateCodec(List inputPaths) throws Except rewriter.close(); // Verify the schema is not changed for the columns not pruned - validateSchema(); + validateSchemaWithGenderColumnPruned(); // Verify codec has been translated FileDecryptionProperties fileDecryptionProperties = EncDecProperties.getFileDecryptionProperties(); @@ -500,14 +484,7 @@ private void testNullifyAndEncryptColumn(List inputPaths) throws Exception validateColumnData(Collections.emptySet(), maskColumns.keySet(), fileDecryptionProperties, false); // Verify the page index - validatePageIndex(new HashMap() { - { - put(1, 1); - put(2, 2); - put(3, 3); - put(4, 4); - } - }); + validatePageIndex(ImmutableSet.of("DocId"), false); // Verify the column is encrypted ParquetMetadata metaData = getFileMetaData(outputFile, fileDecryptionProperties); @@ -590,15 +567,7 @@ public void testMergeTwoFilesOnly() throws Exception { validateColumnData(Collections.emptySet(), Collections.emptySet(), null, false); // Verify the page index - validatePageIndex(new HashMap() { - { - put(0, 0); - put(1, 1); - put(2, 2); - put(3, 3); - put(4, 4); - } - }); + validatePageIndex(new HashSet<>(), false); // Verify original.created.by is preserved validateCreatedBy(); @@ -768,21 +737,23 @@ public void testOneInputFileManyInputFilesToJoin() throws Exception { inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); List inputPathsR = inputFilesToJoin.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList()); - RewriteOptions.Builder builder = createBuilder(inputPathsL, inputPathsR, true); - RewriteOptions options = builder.indexCacheStrategy(indexCacheStrategy).build(); + List pruneColumns = Collections.singletonList("Gender"); + Map maskColumns = ImmutableMap.of("Links.Forward", MaskMode.NULLIFY); + RewriteOptions options = createBuilder(inputPathsL, inputPathsR, true) + .prune(pruneColumns) + .mask(maskColumns) + .transform(CompressionCodecName.ZSTD) + .indexCacheStrategy(indexCacheStrategy) + .overwriteInputWithJoinColumns(true) + .build(); rewriter = new ParquetRewriter(options); rewriter.processBlocks(); rewriter.close(); - MessageType actualSchema = ParquetFileReader.readFooter( - conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) - .getFileMetaData() - .getSchema(); - MessageType expectSchema = createSchema(); Map> inputBloomFilters = allInputBloomFilters(); Map> outputBloomFilters = allOutputBloomFilters(null); - Set schemaRColumns = createSchemaR().getColumns().stream() + Set schemaRColumns = createSchemaRight().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); Set rBloomFilters = outputBloomFilters.keySet().stream() @@ -790,32 +761,16 @@ conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) .collect(Collectors.toSet()); // TODO potentially too many checks, might need to be split into multiple tests - validateColumnData(Collections.emptySet(), Collections.emptySet(), null, true); // Verify data - assertEquals(expectSchema, actualSchema); // Verify schema + validateColumnData(new HashSet<>(pruneColumns), maskColumns.keySet(), null, true); // Verify data + validateSchemaWithGenderColumnPruned(); // Verify schema validateCreatedBy(); // Verify original.created.by assertEquals(inputBloomFilters.keySet(), rBloomFilters); // Verify bloom filters - verifyCodec( // Verify codec - outputFile, - new HashSet() { - { - add(CompressionCodecName.GZIP); - add(CompressionCodecName.UNCOMPRESSED); - } - }, - null); - validatePageIndex( - new HashMap() { // Verify page index - { // verifying only left side input columns - put(0, 0); - put(1, 1); - put(2, 2); - put(3, 4); - } - }); + verifyCodec(outputFile, ImmutableSet.of(CompressionCodecName.ZSTD), null); // Verify codec + validatePageIndex(ImmutableSet.of("Links.Forward"), true); } private void testOneInputFileManyInputFilesToJoinSetup() throws IOException { - inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaL()) + inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaLeft()) .withNumRecord(numRecord) .withRowGroupSize(1 * 1024 * 1024) .withCodec("GZIP") @@ -831,7 +786,7 @@ conf, new Path(inputFiles.get(0).getFileName()), ParquetMetadataConverter.NO_FIL .collect(Collectors.toList()); for (long count : rowGroupRowCounts) { - inputFilesToJoin.add(new TestFileBuilder(conf, createSchemaR()) + inputFilesToJoin.add(new TestFileBuilder(conf, createSchemaRight()) .withNumRecord((int) count) .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) @@ -855,7 +810,7 @@ private MessageType createSchema() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } - private MessageType createSchemaL() { + private MessageType createSchemaLeft() { return new MessageType( "schema", new PrimitiveType(OPTIONAL, INT64, "DocId"), @@ -865,7 +820,7 @@ private MessageType createSchemaL() { new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); } - private MessageType createSchemaR() { + private MessageType createSchemaRight() { return new MessageType( "schema", new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), @@ -1000,77 +955,108 @@ private void verifyCodec( assertEquals(expectedCodecs, codecs); } + @FunctionalInterface + interface CheckedFunction { + R apply(T t) throws IOException; + } + /** * Verify the page index is correct. * - * @param outFileColumnMapping the column mapping from the output file to the input file. + * @param exclude the columns to exclude from comparison, for example because they were nullified. + * @param joinColumnsOverwrite whether a join columns overwrote existing overlapping columns. */ - private void validatePageIndex(Map outFileColumnMapping) throws Exception { - ParquetMetadata outMetaData = getFileMetaData(outputFile, null); - - int inputFileIndex = 0; - TransParquetFileReader inReader = new TransParquetFileReader( - HadoopInputFile.fromPath(new Path(inputFiles.get(inputFileIndex).getFileName()), conf), - HadoopReadOptions.builder(conf).build()); - ParquetMetadata inMetaData = inReader.getFooter(); - - try (TransParquetFileReader outReader = new TransParquetFileReader( - HadoopInputFile.fromPath(new Path(outputFile), conf), - HadoopReadOptions.builder(conf).build())) { - - for (int outBlockId = 0, inBlockId = 0; - outBlockId < outMetaData.getBlocks().size(); - ++outBlockId, ++inBlockId) { - // Refresh reader of input file - if (inBlockId == inMetaData.getBlocks().size()) { - inReader = new TransParquetFileReader( - HadoopInputFile.fromPath( - new Path(inputFiles.get(++inputFileIndex).getFileName()), conf), - HadoopReadOptions.builder(conf).build()); - inMetaData = inReader.getFooter(); - inBlockId = 0; - } + private void validatePageIndex(Set exclude, boolean joinColumnsOverwrite) throws Exception { + class BlockMeta { + final TransParquetFileReader reader; + final BlockMetaData blockMeta; + final Map colPathToMeta; + + BlockMeta( + TransParquetFileReader reader, + BlockMetaData blockMeta, + Map colPathToMeta) { + this.reader = reader; + this.blockMeta = blockMeta; + this.colPathToMeta = colPathToMeta; + } + } - BlockMetaData inBlockMetaData = inMetaData.getBlocks().get(inBlockId); - BlockMetaData outBlockMetaData = outMetaData.getBlocks().get(outBlockId); + CheckedFunction, List> blockMetaExtractor = files -> { + List result = new ArrayList<>(); + for (String inputFile : files) { + TransParquetFileReader reader = new TransParquetFileReader( + HadoopInputFile.fromPath(new Path(inputFile), conf), + HadoopReadOptions.builder(conf).build()); + reader.getFooter() + .getBlocks() + .forEach(x -> result.add(new BlockMeta( + reader, + x, + x.getColumns().stream() + .collect( + Collectors.toMap(ColumnChunkMetaData::getPath, Function.identity()))))); + } + return result; + }; - assertEquals(inBlockMetaData.getRowCount(), outBlockMetaData.getRowCount()); + List inBlocksMain = blockMetaExtractor.apply( + inputFiles.stream().map(EncryptionTestFile::getFileName).collect(Collectors.toList())); + List inBlocksJoined = blockMetaExtractor.apply( + inputFilesToJoin.stream().map(EncryptionTestFile::getFileName).collect(Collectors.toList())); + List outBlocks = blockMetaExtractor.apply(ImmutableList.of(outputFile)); + + for (int blockIdx = 0; blockIdx < outBlocks.size(); blockIdx++) { + BlockMetaData outBlockMeta = outBlocks.get(blockIdx).blockMeta; + TransParquetFileReader outReader = outBlocks.get(blockIdx).reader; + for (ColumnChunkMetaData outChunk : outBlockMeta.getColumns()) { + if (exclude.contains(outChunk.getPath().toDotString())) continue; + TransParquetFileReader inReader; + BlockMetaData inBlockMeta; + ColumnChunkMetaData inChunk; + if (joinColumnsOverwrite + && !inBlocksJoined.isEmpty() + && inBlocksJoined.get(blockIdx).colPathToMeta.containsKey(outChunk.getPath())) { + inReader = inBlocksJoined.get(blockIdx).reader; + inBlockMeta = inBlocksJoined.get(blockIdx).blockMeta; + inChunk = inBlocksJoined.get(blockIdx).colPathToMeta.get(outChunk.getPath()); + } else { + inReader = inBlocksMain.get(blockIdx).reader; + inBlockMeta = inBlocksMain.get(blockIdx).blockMeta; + inChunk = inBlocksMain.get(blockIdx).colPathToMeta.get(outChunk.getPath()); + } - for (int j = 0; j < outBlockMetaData.getColumns().size(); j++) { - if (!outFileColumnMapping.containsKey(j)) { - continue; - } - int columnIdFromInputFile = outFileColumnMapping.get(j); - ColumnChunkMetaData inChunk = inBlockMetaData.getColumns().get(columnIdFromInputFile); - ColumnIndex inColumnIndex = inReader.readColumnIndex(inChunk); - OffsetIndex inOffsetIndex = inReader.readOffsetIndex(inChunk); - ColumnChunkMetaData outChunk = outBlockMetaData.getColumns().get(j); - ColumnIndex outColumnIndex = outReader.readColumnIndex(outChunk); - OffsetIndex outOffsetIndex = outReader.readOffsetIndex(outChunk); - if (inColumnIndex != null) { - assertEquals(inColumnIndex.getBoundaryOrder(), outColumnIndex.getBoundaryOrder()); - assertEquals(inColumnIndex.getMaxValues(), outColumnIndex.getMaxValues()); - assertEquals(inColumnIndex.getMinValues(), outColumnIndex.getMinValues()); - assertEquals(inColumnIndex.getNullCounts(), outColumnIndex.getNullCounts()); - } - if (inOffsetIndex != null) { - List inOffsets = getOffsets(inReader, inChunk); - List outOffsets = getOffsets(outReader, outChunk); - assertEquals(inOffsets.size(), outOffsets.size()); - assertEquals(inOffsets.size(), inOffsetIndex.getPageCount()); - assertEquals(inOffsetIndex.getPageCount(), outOffsetIndex.getPageCount()); - for (int k = 0; k < inOffsetIndex.getPageCount(); k++) { - assertEquals(inOffsetIndex.getFirstRowIndex(k), outOffsetIndex.getFirstRowIndex(k)); - assertEquals( - inOffsetIndex.getLastRowIndex(k, inBlockMetaData.getRowCount()), - outOffsetIndex.getLastRowIndex(k, outBlockMetaData.getRowCount())); - assertEquals(inOffsetIndex.getOffset(k), (long) inOffsets.get(k)); - assertEquals(outOffsetIndex.getOffset(k), (long) outOffsets.get(k)); - } + ColumnIndex inColumnIndex = inReader.readColumnIndex(inChunk); + OffsetIndex inOffsetIndex = inReader.readOffsetIndex(inChunk); + ColumnIndex outColumnIndex = outReader.readColumnIndex(outChunk); + OffsetIndex outOffsetIndex = outReader.readOffsetIndex(outChunk); + if (inColumnIndex != null) { + assertEquals(inColumnIndex.getBoundaryOrder(), outColumnIndex.getBoundaryOrder()); + assertEquals(inColumnIndex.getMaxValues(), outColumnIndex.getMaxValues()); + assertEquals(inColumnIndex.getMinValues(), outColumnIndex.getMinValues()); + assertEquals(inColumnIndex.getNullCounts(), outColumnIndex.getNullCounts()); + } + if (inOffsetIndex != null) { + List inOffsets = getOffsets(inReader, inChunk); + List outOffsets = getOffsets(outReader, outChunk); + assertEquals(inOffsets.size(), outOffsets.size()); + assertEquals(inOffsets.size(), inOffsetIndex.getPageCount()); + assertEquals(inOffsetIndex.getPageCount(), outOffsetIndex.getPageCount()); + for (int k = 0; k < inOffsetIndex.getPageCount(); k++) { + assertEquals(inOffsetIndex.getFirstRowIndex(k), outOffsetIndex.getFirstRowIndex(k)); + assertEquals( + inOffsetIndex.getLastRowIndex(k, inBlockMeta.getRowCount()), + outOffsetIndex.getLastRowIndex(k, outBlockMeta.getRowCount())); + assertEquals(inOffsetIndex.getOffset(k), (long) inOffsets.get(k)); + assertEquals(outOffsetIndex.getOffset(k), (long) outOffsets.get(k)); } } } } + + for (BlockMeta t3 : inBlocksMain) t3.reader.close(); + for (BlockMeta t3 : inBlocksJoined) t3.reader.close(); + for (BlockMeta t3 : outBlocks) t3.reader.close(); } private List getOffsets(TransParquetFileReader reader, ColumnChunkMetaData chunk) throws IOException { @@ -1223,21 +1209,23 @@ private RewriteOptions.Builder createBuilder( return builder; } - private void validateSchema() throws IOException { - ParquetMetadata pmd = - ParquetFileReader.readFooter(conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER); - MessageType schema = pmd.getFileMetaData().getSchema(); - List fields = schema.getFields(); - assertEquals(fields.size(), 5); - assertEquals(fields.get(0).getName(), "DocId"); - assertEquals(fields.get(1).getName(), "Name"); - assertEquals(fields.get(2).getName(), "FloatFraction"); - assertEquals(fields.get(3).getName(), "DoubleFraction"); - assertEquals(fields.get(4).getName(), "Links"); - List subFields = fields.get(4).asGroupType().getFields(); - assertEquals(subFields.size(), 2); - assertEquals(subFields.get(0).getName(), "Backward"); - assertEquals(subFields.get(1).getName(), "Forward"); + private void validateSchemaWithGenderColumnPruned() throws IOException { + MessageType expectSchema = new MessageType( + "schema", + new PrimitiveType(OPTIONAL, INT64, "DocId"), + new PrimitiveType(REQUIRED, BINARY, "Name"), + new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), + new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction"), + new GroupType( + OPTIONAL, + "Links", + new PrimitiveType(REPEATED, BINARY, "Backward"), + new PrimitiveType(REPEATED, BINARY, "Forward"))); + MessageType actualSchema = ParquetFileReader.readFooter( + conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) + .getFileMetaData() + .getSchema(); + assertEquals(expectSchema, actualSchema); } private void ensureContainsGzipFile() { From 8514f396d66bb357886ab721fffa48b62a2613fa Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 26 Aug 2024 22:28:31 -0700 Subject: [PATCH 49/57] add testMergeFilesToJoinWithDifferentRowCount test into ParquetRewriterTest --- .../hadoop/rewrite/ParquetRewriterTest.java | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index b209f44915..1a87f20947 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -729,6 +729,27 @@ private void testSingleInputFileSetup(String... bloomFilterEnabledColumns) throw .build()); } + @Test + public void testMergeFilesToJoinWithDifferentRowCount() throws Exception { + MessageType schema1 = new MessageType("schema", new PrimitiveType(OPTIONAL, INT64, "DocId")); + MessageType schema2 = new MessageType("schema", new PrimitiveType(REQUIRED, BINARY, "Name")); + inputFiles = ImmutableList.of( + new TestFileBuilder(conf, schema1).withNumRecord(numRecord).build()); + inputFilesToJoin = ImmutableList.of( + new TestFileBuilder(conf, schema2).withNumRecord(numRecord / 2).build()); + RewriteOptions.Builder builder = createBuilder( + inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()), + inputFilesToJoin.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()), + true); + RewriteOptions options = builder.build(); + try { + rewriter = + new ParquetRewriter(options); // This should throw an exception because the row count is different + } catch (RuntimeException e) { + assertTrue(e.getMessage().contains("The number of rows in each block must match")); + } + } + @Test public void testOneInputFileManyInputFilesToJoin() throws Exception { testOneInputFileManyInputFilesToJoinSetup(); @@ -981,7 +1002,6 @@ class BlockMeta { this.colPathToMeta = colPathToMeta; } } - CheckedFunction, List> blockMetaExtractor = files -> { List result = new ArrayList<>(); for (String inputFile : files) { @@ -990,10 +1010,10 @@ class BlockMeta { HadoopReadOptions.builder(conf).build()); reader.getFooter() .getBlocks() - .forEach(x -> result.add(new BlockMeta( + .forEach(blockMetaData -> result.add(new BlockMeta( reader, - x, - x.getColumns().stream() + blockMetaData, + blockMetaData.getColumns().stream() .collect( Collectors.toMap(ColumnChunkMetaData::getPath, Function.identity()))))); } @@ -1005,7 +1025,6 @@ class BlockMeta { List inBlocksJoined = blockMetaExtractor.apply( inputFilesToJoin.stream().map(EncryptionTestFile::getFileName).collect(Collectors.toList())); List outBlocks = blockMetaExtractor.apply(ImmutableList.of(outputFile)); - for (int blockIdx = 0; blockIdx < outBlocks.size(); blockIdx++) { BlockMetaData outBlockMeta = outBlocks.get(blockIdx).blockMeta; TransParquetFileReader outReader = outBlocks.get(blockIdx).reader; From 4340c42e3ba2884b1150a485246bc3a7f3161806 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Wed, 28 Aug 2024 22:50:24 -0700 Subject: [PATCH 50/57] add testOneInputFileManyInputFilesToJoin with and without JoinColumnsOverwrite --- .../hadoop/rewrite/ParquetRewriterTest.java | 72 ++++++++++--------- 1 file changed, 37 insertions(+), 35 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 1a87f20947..b35cbf0dec 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -18,10 +18,7 @@ */ package org.apache.parquet.hadoop.rewrite; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; import static org.apache.parquet.schema.Type.Repetition.REPEATED; import static org.apache.parquet.schema.Type.Repetition.REQUIRED; @@ -161,7 +158,7 @@ private void testPruneSingleColumnTranslateCodec(List inputPaths) throws E rewriter.close(); // Verify the schema is not changed for the columns not pruned - validateSchemaWithGenderColumnPruned(); + validateSchemaWithGenderColumnPruned(false); // Verify codec has been translated verifyCodec( @@ -232,7 +229,7 @@ private void testPruneNullifyTranslateCodec(List inputPaths) throws Except rewriter.close(); // Verify the schema are not changed for the columns not pruned - validateSchemaWithGenderColumnPruned(); + validateSchemaWithGenderColumnPruned(false); // Verify codec has been translated verifyCodec( @@ -306,7 +303,7 @@ private void testPruneEncryptTranslateCodec(List inputPaths) throws Except rewriter.close(); // Verify the schema is not changed for the columns not pruned - validateSchemaWithGenderColumnPruned(); + validateSchemaWithGenderColumnPruned(false); // Verify codec has been translated FileDecryptionProperties fileDecryptionProperties = EncDecProperties.getFileDecryptionProperties(); @@ -730,7 +727,7 @@ private void testSingleInputFileSetup(String... bloomFilterEnabledColumns) throw } @Test - public void testMergeFilesToJoinWithDifferentRowCount() throws Exception { + public void testFilesToJoinHaveDifferentRowCount() throws Exception { MessageType schema1 = new MessageType("schema", new PrimitiveType(OPTIONAL, INT64, "DocId")); MessageType schema2 = new MessageType("schema", new PrimitiveType(REQUIRED, BINARY, "Name")); inputFiles = ImmutableList.of( @@ -751,7 +748,16 @@ public void testMergeFilesToJoinWithDifferentRowCount() throws Exception { } @Test - public void testOneInputFileManyInputFilesToJoin() throws Exception { + public void testOneInputFileManyInputFilesToJoinWithJoinColumnsOverwrite() throws Exception { + testOneInputFileManyInputFilesToJoinSetup(true); + } + + @Test + public void testOneInputFileManyInputFilesToJoinWithoutJoinColumnsOverwrite() throws Exception { + testOneInputFileManyInputFilesToJoinSetup(false); + } + + public void testOneInputFileManyInputFilesToJoinSetup(boolean joinColumnsOverwrite) throws Exception { testOneInputFileManyInputFilesToJoinSetup(); List inputPathsL = @@ -765,7 +771,7 @@ public void testOneInputFileManyInputFilesToJoin() throws Exception { .mask(maskColumns) .transform(CompressionCodecName.ZSTD) .indexCacheStrategy(indexCacheStrategy) - .overwriteInputWithJoinColumns(true) + .overwriteInputWithJoinColumns(joinColumnsOverwrite) .build(); rewriter = new ParquetRewriter(options); @@ -774,24 +780,24 @@ public void testOneInputFileManyInputFilesToJoin() throws Exception { Map> inputBloomFilters = allInputBloomFilters(); Map> outputBloomFilters = allOutputBloomFilters(null); - Set schemaRColumns = createSchemaRight().getColumns().stream() + Set schemaRColumns = createSchemaToJoin().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); Set rBloomFilters = outputBloomFilters.keySet().stream() .filter(schemaRColumns::contains) .collect(Collectors.toSet()); - // TODO potentially too many checks, might need to be split into multiple tests - validateColumnData(new HashSet<>(pruneColumns), maskColumns.keySet(), null, true); // Verify data - validateSchemaWithGenderColumnPruned(); // Verify schema + validateColumnData( + new HashSet<>(pruneColumns), maskColumns.keySet(), null, joinColumnsOverwrite); // Verify data + validateSchemaWithGenderColumnPruned(true); // Verify schema validateCreatedBy(); // Verify original.created.by assertEquals(inputBloomFilters.keySet(), rBloomFilters); // Verify bloom filters verifyCodec(outputFile, ImmutableSet.of(CompressionCodecName.ZSTD), null); // Verify codec - validatePageIndex(ImmutableSet.of("Links.Forward"), true); + validatePageIndex(ImmutableSet.of("Links.Forward"), joinColumnsOverwrite); } private void testOneInputFileManyInputFilesToJoinSetup() throws IOException { - inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchemaLeft()) + inputFiles = Lists.newArrayList(new TestFileBuilder(conf, createSchema()) .withNumRecord(numRecord) .withRowGroupSize(1 * 1024 * 1024) .withCodec("GZIP") @@ -807,7 +813,7 @@ conf, new Path(inputFiles.get(0).getFileName()), ParquetMetadataConverter.NO_FIL .collect(Collectors.toList()); for (long count : rowGroupRowCounts) { - inputFilesToJoin.add(new TestFileBuilder(conf, createSchemaRight()) + inputFilesToJoin.add(new TestFileBuilder(conf, createSchemaToJoin()) .withNumRecord((int) count) .withCodec("UNCOMPRESSED") .withPageSize(ParquetProperties.DEFAULT_PAGE_SIZE) @@ -831,20 +837,11 @@ private MessageType createSchema() { new PrimitiveType(REPEATED, BINARY, "Forward"))); } - private MessageType createSchemaLeft() { - return new MessageType( - "schema", - new PrimitiveType(OPTIONAL, INT64, "DocId"), - new PrimitiveType(REQUIRED, BINARY, "Name"), - new PrimitiveType(OPTIONAL, BINARY, "Gender"), - new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), - new PrimitiveType(OPTIONAL, DOUBLE, "DoubleFraction")); - } - - private MessageType createSchemaRight() { + private MessageType createSchemaToJoin() { return new MessageType( "schema", new PrimitiveType(REPEATED, FLOAT, "FloatFraction"), + new PrimitiveType(OPTIONAL, INT64, "Age"), new GroupType( OPTIONAL, "Links", @@ -870,9 +867,10 @@ private void validateColumnData( .flatMap(x -> Arrays.stream(x.getFileContent())) .collect(Collectors.toList()); BiFunction groups = (name, rowIdx) -> { - if (joinColumnsOverwrite - && !filesJoined.isEmpty() - && filesJoined.get(0).getType().containsField(name)) { + if (!filesMain.get(0).getType().containsField(name) + || joinColumnsOverwrite + && !filesJoined.isEmpty() + && filesJoined.get(0).getType().containsField(name)) { return filesJoined.get(rowIdx); } else { return filesMain.get(rowIdx); @@ -1033,9 +1031,10 @@ class BlockMeta { TransParquetFileReader inReader; BlockMetaData inBlockMeta; ColumnChunkMetaData inChunk; - if (joinColumnsOverwrite - && !inBlocksJoined.isEmpty() - && inBlocksJoined.get(blockIdx).colPathToMeta.containsKey(outChunk.getPath())) { + if (!inBlocksMain.get(blockIdx).colPathToMeta.containsKey(outChunk.getPath()) + || joinColumnsOverwrite + && !inBlocksJoined.isEmpty() + && inBlocksJoined.get(blockIdx).colPathToMeta.containsKey(outChunk.getPath())) { inReader = inBlocksJoined.get(blockIdx).reader; inBlockMeta = inBlocksJoined.get(blockIdx).blockMeta; inChunk = inBlocksJoined.get(blockIdx).colPathToMeta.get(outChunk.getPath()); @@ -1228,7 +1227,7 @@ private RewriteOptions.Builder createBuilder( return builder; } - private void validateSchemaWithGenderColumnPruned() throws IOException { + private void validateSchemaWithGenderColumnPruned(boolean addJoinedColumn) throws IOException { MessageType expectSchema = new MessageType( "schema", new PrimitiveType(OPTIONAL, INT64, "DocId"), @@ -1240,6 +1239,9 @@ private void validateSchemaWithGenderColumnPruned() throws IOException { "Links", new PrimitiveType(REPEATED, BINARY, "Backward"), new PrimitiveType(REPEATED, BINARY, "Forward"))); + if (addJoinedColumn) { + expectSchema = expectSchema.union(new MessageType("schema", new PrimitiveType(OPTIONAL, INT64, "Age"))); + } MessageType actualSchema = ParquetFileReader.readFooter( conf, new Path(outputFile), ParquetMetadataConverter.NO_FILTER) .getFileMetaData() From bb42979f0912c2216ffbe3b368c68b02a8b6f23e Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sat, 31 Aug 2024 11:27:06 -0700 Subject: [PATCH 51/57] add encrypt validation into ParquetRewriterTest's testOneInputFileManyInputFilesToJoin --- .../hadoop/rewrite/ParquetRewriterTest.java | 37 ++++++++++++++++--- 1 file changed, 31 insertions(+), 6 deletions(-) diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index b35cbf0dec..8b799afc8e 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -760,18 +760,27 @@ public void testOneInputFileManyInputFilesToJoinWithoutJoinColumnsOverwrite() th public void testOneInputFileManyInputFilesToJoinSetup(boolean joinColumnsOverwrite) throws Exception { testOneInputFileManyInputFilesToJoinSetup(); + String encryptColumn = "DocId"; + String pruneColumn = "Gender"; + + FileEncryptionProperties fileEncryptionProperties = EncDecProperties.getFileEncryptionProperties( + new String[] {encryptColumn}, ParquetCipher.AES_GCM_CTR_V1, false); + FileDecryptionProperties fileDecryptionProperties = EncDecProperties.getFileDecryptionProperties(); + List inputPathsL = inputFiles.stream().map(x -> new Path(x.getFileName())).collect(Collectors.toList()); List inputPathsR = inputFilesToJoin.stream().map(y -> new Path(y.getFileName())).collect(Collectors.toList()); - List pruneColumns = Collections.singletonList("Gender"); - Map maskColumns = ImmutableMap.of("Links.Forward", MaskMode.NULLIFY); + List pruneColumns = ImmutableList.of(pruneColumn); + Map maskColumns = ImmutableMap.of(encryptColumn, MaskMode.NULLIFY); RewriteOptions options = createBuilder(inputPathsL, inputPathsR, true) .prune(pruneColumns) .mask(maskColumns) .transform(CompressionCodecName.ZSTD) .indexCacheStrategy(indexCacheStrategy) .overwriteInputWithJoinColumns(joinColumnsOverwrite) + .encrypt(ImmutableList.of(encryptColumn)) + .encryptionProperties(fileEncryptionProperties) .build(); rewriter = new ParquetRewriter(options); @@ -779,7 +788,7 @@ public void testOneInputFileManyInputFilesToJoinSetup(boolean joinColumnsOverwri rewriter.close(); Map> inputBloomFilters = allInputBloomFilters(); - Map> outputBloomFilters = allOutputBloomFilters(null); + Map> outputBloomFilters = allOutputBloomFilters(fileDecryptionProperties); Set schemaRColumns = createSchemaToJoin().getColumns().stream() .map(x -> ColumnPath.get(x.getPath())) .collect(Collectors.toSet()); @@ -787,13 +796,29 @@ public void testOneInputFileManyInputFilesToJoinSetup(boolean joinColumnsOverwri .filter(schemaRColumns::contains) .collect(Collectors.toSet()); + // Verify column encryption + ParquetMetadata metaData = getFileMetaData(outputFile, fileDecryptionProperties); + assertFalse(metaData.getBlocks().isEmpty()); + List columns = metaData.getBlocks().get(0).getColumns(); + Set set = ImmutableSet.of(encryptColumn); + for (ColumnChunkMetaData column : columns) { + if (set.contains(column.getPath().toDotString())) { + assertTrue(column.isEncrypted()); + } else { + assertFalse(column.isEncrypted()); + } + } + validateColumnData( - new HashSet<>(pruneColumns), maskColumns.keySet(), null, joinColumnsOverwrite); // Verify data + new HashSet<>(pruneColumns), + maskColumns.keySet(), + fileDecryptionProperties, + joinColumnsOverwrite); // Verify data validateSchemaWithGenderColumnPruned(true); // Verify schema validateCreatedBy(); // Verify original.created.by assertEquals(inputBloomFilters.keySet(), rBloomFilters); // Verify bloom filters - verifyCodec(outputFile, ImmutableSet.of(CompressionCodecName.ZSTD), null); // Verify codec - validatePageIndex(ImmutableSet.of("Links.Forward"), joinColumnsOverwrite); + verifyCodec(outputFile, ImmutableSet.of(CompressionCodecName.ZSTD), fileDecryptionProperties); // Verify codec + validatePageIndex(ImmutableSet.of(encryptColumn), joinColumnsOverwrite); } private void testOneInputFileManyInputFilesToJoinSetup() throws IOException { From 5b97a4c5d76fa20b2d561f68e9bc7f68a7bff5cc Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 8 Sep 2024 14:33:49 -0700 Subject: [PATCH 52/57] refactor ParquetRewriter slightly to address PR comments --- .../parquet/hadoop/rewrite/ParquetRewriter.java | 4 +--- .../hadoop/rewrite/ParquetRewriterTest.java | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 1258568941..4bfb4a49f1 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -133,8 +133,7 @@ public ParquetRewriter(RewriteOptions options) throws IOException { .collect(Collectors.toList()), out); - this.outSchema = getSchema(); - this.outSchema = pruneColumnsInSchema(outSchema, options.getPruneColumns()); + this.outSchema = pruneColumnsInSchema(getSchema(), options.getPruneColumns()); this.extraMetaData = getExtraMetadata(options); if (options.getMaskColumns() != null) { @@ -208,7 +207,6 @@ private MessageType getSchema() { .getFields() .forEach(x -> { if (!fieldNames.containsKey(x.getName())) { - LOG.info("Column {} is added to the output from inputFilesToJoin side", x.getName()); fieldNames.put(x.getName(), x); } else if (overwriteInputWithJoinColumns) { LOG.info("Column {} in inputFiles is overwritten by inputFilesToJoin side", x.getName()); diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java index 8b799afc8e..ce612baec9 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/rewrite/ParquetRewriterTest.java @@ -18,7 +18,10 @@ */ package org.apache.parquet.hadoop.rewrite; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; import static org.apache.parquet.schema.Type.Repetition.REPEATED; import static org.apache.parquet.schema.Type.Repetition.REQUIRED; @@ -36,7 +39,14 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; From 27ba73b6fc047fac20f7a446c8a330b713842af9 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Sun, 8 Sep 2024 17:00:07 -0700 Subject: [PATCH 53/57] add javadoc to ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 16 +++++++ .../hadoop/rewrite/RewriteOptions.java | 44 ++++++++++++------- 2 files changed, 44 insertions(+), 16 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 4bfb4a49f1..8dec656910 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -90,6 +90,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Rewrites multiple input files with the same schema into a single output file. Optionally allows to: 1) apply + * column transformations; 2) join with extra files with a different schema. + *

+ * Some supported column transformations: pruning, masking, encrypting, changing a codec. + * See {@link RewriteOptions} and {@link RewriteOptions.Builder} for the full list with description. + *

+ * Requirements for a joining the main input files(left) and extra input files(right): + * 1) the number of files might be different on the left and right, + * 2) the schema might be different on the left and right, + * 3) the total number of row groups must be the same on the left and right, + * 4) the total number of rows must be the same on the left and right, + * 5) the global ordering of rows must be the same on the left and right. + */ public class ParquetRewriter implements Closeable { // Key to store original writer version in the file key-value metadata @@ -162,6 +176,8 @@ public ParquetRewriter(RewriteOptions options) throws IOException { writer.start(); } + // TODO: Should we mark it as deprecated to encourage the main constructor usage? it is also used only from + // deprecated classes atm // Ctor for legacy CompressionConverter and ColumnMasker public ParquetRewriter( TransParquetFileReader reader, diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 82ef2f8086..9211f7c340 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -38,15 +38,7 @@ import org.apache.parquet.io.OutputFile; /** - * A set of options to create a ParquetRewriter. - * - * TODO find a place where to put a proper description of functionality as it is not trivial: - * ParquetRewriter allows to stitch files with the same schema into a single file. - * Note that ParquetRewriter also can be used for effectively stitch/joining multiple parquet files with - * different schemas. - * You can provide the main input file group and multiple right side ones. That is possible when: - * 1) the number of rows in the main and extra input groups are the same, - * 2) the ordering of rows in the main and extra input groups is the same. + * A set of options to create a {@link ParquetRewriter}. See {@link RewriteOptions.Builder} for options description. */ public class RewriteOptions { @@ -127,6 +119,26 @@ public List getInputFiles() { .collect(Collectors.toList()); } + /** + * Gets the input {@link Path}s for the rewrite if they exist for all input files to join, + * otherwise throws a {@link RuntimeException}. + * + * @return a {@link List} of the associated input {@link Path}s to join + */ + public List getInputFilesToJoin() { + return inputFilesToJoin.stream() + .map(f -> { + if (f instanceof HadoopOutputFile) { + HadoopOutputFile hadoopOutputFile = (HadoopOutputFile) f; + return new Path(hadoopOutputFile.getPath()); + } else { + throw new RuntimeException( + "The input files to join do not all have an associated Hadoop Path."); + } + }) + .collect(Collectors.toList()); + } + /** * Gets the {@link InputFile}s for the rewrite. * @@ -136,10 +148,10 @@ public List getParquetInputFiles() { return inputFiles; } - /** TODO fix documentation after addition of InputFilesToJoin - * Gets the right {@link InputFile}s for the rewrite. + /** + * Gets the right {@link InputFile}s to join during the rewrite. * - * @return a {@link List} of the associated right {@link InputFile}s + * @return a {@link List} of the associated {@link InputFile}s to join */ public List getParquetInputFilesToJoin() { return inputFilesToJoin; @@ -200,7 +212,7 @@ public boolean getIgnoreJoinFilesMetadata() { return ignoreJoinFilesMetadata; } - // Builder to create a RewriterOptions. + /** Builder for {@link RewriteOptions} which in turn is used in {@link ParquetRewriter}'s constructor. */ public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; @@ -412,7 +424,7 @@ public Builder addInputFile(InputFile inputFile) { } /** - * Add a file to join to other input files. + * Add an input file to join. * * @param fileToJoin input file to join * @return self @@ -437,7 +449,7 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { } /** - * Set a flag whether columns from join files need to overwrite columns from input files. + * Set a flag whether columns from join files need to overwrite columns from the main input files. * * @param overwriteInputWithJoinColumns * @return self @@ -448,7 +460,7 @@ public Builder overwriteInputWithJoinColumns(boolean overwriteInputWithJoinColum } /** - * Set a flag whether metadata from join files should be ignored. + * Set a flag whether metadata from join files should be ignored, false by default. * * @param ignoreJoinFilesMetadata * @return self From 07f1e74d851d8ab41c574ac55b86f28e8ee8bdad Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Mon, 9 Sep 2024 17:44:39 -0700 Subject: [PATCH 54/57] add javadoc to ParquetRewriter --- .../hadoop/rewrite/ParquetRewriter.java | 36 ++++++-- .../hadoop/rewrite/RewriteOptions.java | 89 +++++++++++++++++-- 2 files changed, 110 insertions(+), 15 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 8dec656910..d5ca16a7ba 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -91,18 +91,38 @@ import org.slf4j.LoggerFactory; /** - * Rewrites multiple input files with the same schema into a single output file. Optionally allows to: 1) apply - * column transformations; 2) join with extra files with a different schema. + * Rewrites multiple input files into a single output file. *

+ * Supported functionality: + *

    + *
  • Merging multiple files into a single one
  • + *
  • Applying column transformations
  • + *
  • Joining with extra files with a different schema
  • + *
+ *

+ * Note that the total number of row groups from all input files is preserved in the output file. + * This may not be optimal if row groups are very small and will not solve small file problems. Instead, it will + * make it worse to have a large file footer in the output file. + *

+ *

Merging multiple files into a single output files

+ * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFiles. + * Please note the schema of all inputFiles must be the same, otherwise the rewrite will fail. + *

+ *

Applying column transformations

* Some supported column transformations: pruning, masking, encrypting, changing a codec. * See {@link RewriteOptions} and {@link RewriteOptions.Builder} for the full list with description. *

- * Requirements for a joining the main input files(left) and extra input files(right): - * 1) the number of files might be different on the left and right, - * 2) the schema might be different on the left and right, - * 3) the total number of row groups must be the same on the left and right, - * 4) the total number of rows must be the same on the left and right, - * 5) the global ordering of rows must be the same on the left and right. + *

Joining with extra files with a different schema.

+ * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFilesToJoin. + * Please note the schema of all inputFilesToJoin must be the same, otherwise the rewrite will fail. + * Requirements for a joining the main inputFiles(left) and inputFilesToJoin(right): + *
    + *
  • the number of files might be different on the left and right,
  • + *
  • the schema of files inside of each group(left/right) must be the same, but those two schemas not necessarily should be equal,
  • + *
  • the total number of row groups must be the same on the left and right,
  • + *
  • the total number of rows must be the same on the left and right,
  • + *
  • the global ordering of rows must be the same on the left and right.
  • + *
*/ public class ParquetRewriter implements Closeable { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java index 9211f7c340..a69403f464 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java @@ -212,7 +212,7 @@ public boolean getIgnoreJoinFilesMetadata() { return ignoreJoinFilesMetadata; } - /** Builder for {@link RewriteOptions} which in turn is used in {@link ParquetRewriter}'s constructor. */ + /** Builder for {@link RewriteOptions} which is used for constructing {@link ParquetRewriter}.*/ public static class Builder { private final ParquetConfiguration conf; private final List inputFiles; @@ -227,6 +227,22 @@ public static class Builder { private boolean overwriteInputWithJoinColumns = false; private boolean ignoreJoinFilesMetadata = false; + /** + * Create a builder to create a RewriterOptions. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFile input file path to read from + * @param inputFileToJoin input join file path to read from + * @param outputFile output file path to rewrite to + */ + public Builder(Configuration conf, Path inputFile, Path inputFileToJoin, Path outputFile) { + this( + new HadoopParquetConfiguration(conf), + HadoopInputFile.fromPathUnchecked(inputFile, conf), + HadoopInputFile.fromPathUnchecked(inputFileToJoin, conf), + HadoopOutputFile.fromPathUnchecked(outputFile, conf)); + } + /** * Create a builder to create a RewriterOptions. * @@ -249,7 +265,20 @@ public Builder(Configuration conf, Path inputFile, Path outputFile) { * @param outputFile output file to rewrite to */ public Builder(ParquetConfiguration conf, InputFile inputFile, OutputFile outputFile) { - this(conf, Collections.singletonList(inputFile), outputFile); + this(conf, Collections.singletonList(inputFile), null, outputFile); + } + + /** + * Create a builder to create a RewriterOptions. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFile input file to read from + * @param inputFileToJoin input join file to read from + * @param outputFile output file to rewrite to + */ + public Builder( + ParquetConfiguration conf, InputFile inputFile, InputFile inputFileToJoin, OutputFile outputFile) { + this(conf, Collections.singletonList(inputFile), Collections.singletonList(inputFileToJoin), outputFile); } /** @@ -262,6 +291,8 @@ public Builder(ParquetConfiguration conf, InputFile inputFile, OutputFile output * if row groups are very small and will not solve small file problems. Instead, it will * make it worse to have a large file footer in the output file. * TODO: support rewrite by record to break the original row groups into reasonable ones. + *

+ * See {@link ParquetRewriter} for more details. * * @param conf configuration for reading from input files and writing to output file * @param inputFiles list of input file paths to read from @@ -287,6 +318,8 @@ public Builder(Configuration conf, List inputFiles, Path outputFile) { * if row groups are very small and will not solve small file problems. Instead, it will * make it worse to have a large file footer in the output file. * TODO: support rewrite by record to break the original row groups into reasonable ones. + *

+ * See {@link ParquetRewriter} for more details. * * @param conf configuration for reading from input files and writing to output file * @param inputFiles list of input file paths to read from @@ -299,6 +332,25 @@ public Builder(ParquetConfiguration conf, List inputFiles, OutputFile this.outputFile = outputFile; } + /** + * Create a builder to create a RewriterOptions. + *

+ * Please note the schema of all files in each file group inputFiles and inputFilesToJoin + * must be the same while those two schemas can be different in comparison with each other. + * Otherwise, the rewrite will fail. + *

+ * The rewrite will keep original row groups from all input files. This may not be optimal + * if row groups are very small and will not solve small file problems. Instead, it will + * make it worse to have a large file footer in the output file. + * TODO: support rewrite by record to break the original row groups into reasonable ones. + *

+ * See {@link ParquetRewriter} for more details. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFiles list of input file paths to read from + * @param inputFilesToJoin list of input join file paths to read from + * @param outputFile output file path to rewrite to + */ public Builder(Configuration conf, List inputFiles, List inputFilesToJoin, Path outputFile) { this.conf = new HadoopParquetConfiguration(conf); this.inputFiles = new ArrayList<>(inputFiles.size()); @@ -312,6 +364,24 @@ public Builder(Configuration conf, List inputFiles, List inputFilesT this.outputFile = HadoopOutputFile.fromPathUnchecked(outputFile, conf); } + /** + * Create a builder to create a RewriterOptions. + *

+ * Please note the schema of all files in each file group inputFiles and inputFilesToJoin + * must be the same while those two schemas can be different in comparison with each other. + * Otherwise, the rewrite will fail. + *

+ * The rewrite will keep original row groups from all input files. This may not be optimal + * if row groups are very small and will not solve small file problems. Instead, it will + * make it worse to have a large file footer in the output file. + *

+ * See {@link ParquetRewriter} for more details. + * + * @param conf configuration for reading from input files and writing to output file + * @param inputFiles list of input file paths to read from + * @param inputFilesToJoin list of input join file paths to read from + * @param outputFile output file path to rewrite to + */ public Builder( ParquetConfiguration conf, List inputFiles, @@ -400,8 +470,8 @@ public Builder addInputFile(Path path) { return this; } - /** TODO fix documentation after addition of InputFilesToJoin - * Add an input file to read from. + /** + * Add an input join file to read from. * * @param path input file path to read from * @return self @@ -450,8 +520,11 @@ public Builder indexCacheStrategy(IndexCache.CacheStrategy cacheStrategy) { /** * Set a flag whether columns from join files need to overwrite columns from the main input files. + *

+ * By default, join files columns do not overwrite the main input file columns. * - * @param overwriteInputWithJoinColumns + * @param overwriteInputWithJoinColumns a flag if columns from join files should overwrite columns + * from the main input files * @return self */ public Builder overwriteInputWithJoinColumns(boolean overwriteInputWithJoinColumns) { @@ -460,9 +533,11 @@ public Builder overwriteInputWithJoinColumns(boolean overwriteInputWithJoinColum } /** - * Set a flag whether metadata from join files should be ignored, false by default. + * Set a flag whether metadata from join files should be ignored. + *

+ * By default, metadata is not ignored. * - * @param ignoreJoinFilesMetadata + * @param ignoreJoinFilesMetadata a flag if metadata from join files should be ignored * @return self */ public Builder ignoreJoinFilesMetadata(boolean ignoreJoinFilesMetadata) { From e96c022c71ab8ecdbdf3dffe3045734259a3715d Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 12 Sep 2024 19:42:33 -0700 Subject: [PATCH 55/57] fix javadoc in ParquetRewriter to comply with Maven javadoc plugin --- .../apache/parquet/hadoop/rewrite/ParquetRewriter.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index d5ca16a7ba..14350d4c8c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -92,7 +92,7 @@ /** * Rewrites multiple input files into a single output file. - *

+ *

* Supported functionality: *
    *
  • Merging multiple files into a single one
  • @@ -104,15 +104,15 @@ * This may not be optimal if row groups are very small and will not solve small file problems. Instead, it will * make it worse to have a large file footer in the output file. *

    - *

    Merging multiple files into a single output files

    + *

    Merging multiple files into a single output files

    * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFiles. * Please note the schema of all inputFiles must be the same, otherwise the rewrite will fail. *

    - *

    Applying column transformations

    + *

    Applying column transformations

    * Some supported column transformations: pruning, masking, encrypting, changing a codec. * See {@link RewriteOptions} and {@link RewriteOptions.Builder} for the full list with description. *

    - *

    Joining with extra files with a different schema.

    + *

    Joining with extra files with a different schema

    * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFilesToJoin. * Please note the schema of all inputFilesToJoin must be the same, otherwise the rewrite will fail. * Requirements for a joining the main inputFiles(left) and inputFilesToJoin(right): From d1c1d765ad084dd14b2680205bd36a2c33a44e7b Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 12 Sep 2024 20:35:04 -0700 Subject: [PATCH 56/57] fix javadoc in ParquetRewriter to comply with Maven javadoc plugin --- .../org/apache/parquet/hadoop/rewrite/ParquetRewriter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 14350d4c8c..904a88b30e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -104,15 +104,15 @@ * This may not be optimal if row groups are very small and will not solve small file problems. Instead, it will * make it worse to have a large file footer in the output file. *

    - *

    Merging multiple files into a single output files

    + *

    Merging multiple files into a single output files

    * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFiles. * Please note the schema of all inputFiles must be the same, otherwise the rewrite will fail. *

    - *

    Applying column transformations

    + *

    Applying column transformations

    * Some supported column transformations: pruning, masking, encrypting, changing a codec. * See {@link RewriteOptions} and {@link RewriteOptions.Builder} for the full list with description. *

    - *

    Joining with extra files with a different schema

    + *

    Joining with extra files with a different schema

    * Use {@link RewriteOptions.Builder}'s constructor or methods to provide inputFilesToJoin. * Please note the schema of all inputFilesToJoin must be the same, otherwise the rewrite will fail. * Requirements for a joining the main inputFiles(left) and inputFilesToJoin(right): From 9de20d77991cafca6239b430a4b9966657bc9a37 Mon Sep 17 00:00:00 2001 From: maxim_konstantinov Date: Thu, 12 Sep 2024 21:15:37 -0700 Subject: [PATCH 57/57] fix javadoc in ParquetRewriter to comply with Maven javadoc plugin --- .../java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java index 904a88b30e..9e106fc3cb 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java @@ -92,7 +92,7 @@ /** * Rewrites multiple input files into a single output file. - *

    + *

    * Supported functionality: *

      *
    • Merging multiple files into a single one