-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Implement per column compression #3396
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,104 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.parquet.column; | ||
|
|
||
| import static org.junit.Assert.assertEquals; | ||
|
|
||
| import org.apache.parquet.hadoop.metadata.CompressionCodecName; | ||
| import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; | ||
| import org.junit.Test; | ||
|
|
||
| public class ParquetPropertiesCompressionTest { | ||
|
|
||
| private static ColumnDescriptor col(String name) { | ||
| return new ColumnDescriptor(new String[] {name}, PrimitiveTypeName.BINARY, 0, 0); | ||
| } | ||
|
|
||
| @Test | ||
| public void testDefaultCompressionCodecIsUncompressed() { | ||
| ParquetProperties props = ParquetProperties.builder().build(); | ||
| assertEquals(CompressionCodecName.UNCOMPRESSED, props.getCompressionCodec(col("any_column"))); | ||
| assertEquals(CompressionCodecName.UNCOMPRESSED, props.getDefaultCompressionCodec()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSetDefaultCompressionCodec() { | ||
| ParquetProperties props = ParquetProperties.builder() | ||
| .withCompressionCodec(CompressionCodecName.SNAPPY) | ||
| .build(); | ||
| assertEquals(CompressionCodecName.SNAPPY, props.getCompressionCodec(col("any_column"))); | ||
| assertEquals(CompressionCodecName.SNAPPY, props.getDefaultCompressionCodec()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testPerColumnCompressionCodec() { | ||
| ParquetProperties props = ParquetProperties.builder() | ||
| .withCompressionCodec(CompressionCodecName.SNAPPY) | ||
| .withCompressionCodec("col_a", CompressionCodecName.GZIP) | ||
| .withCompressionCodec("col_b", CompressionCodecName.UNCOMPRESSED) | ||
| .build(); | ||
|
|
||
| // Per-column overrides | ||
| assertEquals(CompressionCodecName.GZIP, props.getCompressionCodec(col("col_a"))); | ||
| assertEquals(CompressionCodecName.UNCOMPRESSED, props.getCompressionCodec(col("col_b"))); | ||
| // Default for non-overridden columns | ||
| assertEquals(CompressionCodecName.SNAPPY, props.getCompressionCodec(col("col_c"))); | ||
| assertEquals(CompressionCodecName.SNAPPY, props.getDefaultCompressionCodec()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testCopyPreservesCompressionCodec() { | ||
| ParquetProperties original = ParquetProperties.builder() | ||
| .withCompressionCodec(CompressionCodecName.GZIP) | ||
| .withCompressionCodec("col_a", CompressionCodecName.SNAPPY) | ||
| .build(); | ||
|
|
||
| ParquetProperties copy = ParquetProperties.copy(original).build(); | ||
|
|
||
| assertEquals(CompressionCodecName.GZIP, copy.getCompressionCodec(col("other"))); | ||
| assertEquals(CompressionCodecName.SNAPPY, copy.getCompressionCodec(col("col_a"))); | ||
| } | ||
|
|
||
| @Test | ||
| public void testCopyCanOverrideDefault() { | ||
| ParquetProperties original = ParquetProperties.builder() | ||
| .withCompressionCodec(CompressionCodecName.GZIP) | ||
| .withCompressionCodec("col_a", CompressionCodecName.SNAPPY) | ||
| .build(); | ||
|
|
||
| ParquetProperties modified = ParquetProperties.copy(original) | ||
| .withCompressionCodec(CompressionCodecName.ZSTD) | ||
| .build(); | ||
|
|
||
| // Default overridden | ||
| assertEquals(CompressionCodecName.ZSTD, modified.getCompressionCodec(col("other"))); | ||
| // Per-column override preserved | ||
| assertEquals(CompressionCodecName.SNAPPY, modified.getCompressionCodec(col("col_a"))); | ||
| } | ||
|
|
||
| @Test(expected = NullPointerException.class) | ||
| public void testNullDefaultCompressionCodecThrows() { | ||
| ParquetProperties.builder().withCompressionCodec((CompressionCodecName) null); | ||
| } | ||
|
|
||
| @Test(expected = NullPointerException.class) | ||
| public void testNullPerColumnCompressionCodecThrows() { | ||
| ParquetProperties.builder().withCompressionCodec("col_a", null); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.function.Function; | ||
| import java.util.zip.CRC32; | ||
| import org.apache.parquet.bytes.ByteBufferAllocator; | ||
| import org.apache.parquet.bytes.ByteBufferReleaser; | ||
|
|
@@ -44,6 +45,7 @@ | |
| import org.apache.parquet.column.values.bloomfilter.BloomFilter; | ||
| import org.apache.parquet.column.values.bloomfilter.BloomFilterWriteStore; | ||
| import org.apache.parquet.column.values.bloomfilter.BloomFilterWriter; | ||
| import org.apache.parquet.compression.CompressionCodecFactory; | ||
| import org.apache.parquet.compression.CompressionCodecFactory.BytesInputCompressor; | ||
| import org.apache.parquet.crypto.AesCipher; | ||
| import org.apache.parquet.crypto.InternalColumnEncryptionSetup; | ||
|
|
@@ -576,22 +578,7 @@ public ColumnChunkPageWriteStore( | |
| ByteBufferAllocator allocator, | ||
| int columnIndexTruncateLength, | ||
| boolean pageWriteChecksumEnabled) { | ||
| this.schema = schema; | ||
| for (ColumnDescriptor path : schema.getColumns()) { | ||
| writers.put( | ||
| path, | ||
| new ColumnChunkPageWriter( | ||
| path, | ||
| compressor, | ||
| allocator, | ||
| columnIndexTruncateLength, | ||
| pageWriteChecksumEnabled, | ||
| null, | ||
| null, | ||
| null, | ||
| -1, | ||
| -1)); | ||
| } | ||
| this(compressor, schema, allocator, columnIndexTruncateLength, pageWriteChecksumEnabled, null, 0); | ||
| } | ||
|
|
||
| @Deprecated | ||
|
|
@@ -621,14 +608,32 @@ public ColumnChunkPageWriteStore( | |
| boolean pageWriteChecksumEnabled, | ||
| InternalFileEncryptor fileEncryptor, | ||
| int rowGroupOrdinal) { | ||
| this( | ||
| path -> compressor, | ||
| schema, | ||
| allocator, | ||
| columnIndexTruncateLength, | ||
| pageWriteChecksumEnabled, | ||
| fileEncryptor, | ||
| rowGroupOrdinal); | ||
| } | ||
|
|
||
| private ColumnChunkPageWriteStore( | ||
| Function<ColumnDescriptor, BytesInputCompressor> compressorProvider, | ||
| MessageType schema, | ||
| ByteBufferAllocator allocator, | ||
| int columnIndexTruncateLength, | ||
| boolean pageWriteChecksumEnabled, | ||
| InternalFileEncryptor fileEncryptor, | ||
| int rowGroupOrdinal) { | ||
| this.schema = schema; | ||
| if (null == fileEncryptor) { | ||
| for (ColumnDescriptor path : schema.getColumns()) { | ||
| writers.put( | ||
| path, | ||
| new ColumnChunkPageWriter( | ||
| path, | ||
| compressor, | ||
| compressorProvider.apply(path), | ||
| allocator, | ||
| columnIndexTruncateLength, | ||
| pageWriteChecksumEnabled, | ||
|
|
@@ -660,7 +665,7 @@ public ColumnChunkPageWriteStore( | |
| path, | ||
| new ColumnChunkPageWriter( | ||
| path, | ||
| compressor, | ||
| compressorProvider.apply(path), | ||
| allocator, | ||
| columnIndexTruncateLength, | ||
| pageWriteChecksumEnabled, | ||
|
|
@@ -672,6 +677,88 @@ public ColumnChunkPageWriteStore( | |
| } | ||
| } | ||
|
|
||
| public static Builder builder() { | ||
| return new Builder(); | ||
| } | ||
|
|
||
| /** | ||
| * Builder for {@link ColumnChunkPageWriteStore}. Prefer this over the constructors when new | ||
| * parameters are needed so that callers do not have to be updated every time a parameter is | ||
| * added. | ||
| */ | ||
| public static class Builder { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does this new builder look good to you? @gszadovszky @Fokko? |
||
| private Function<ColumnDescriptor, BytesInputCompressor> compressorProvider; | ||
| private MessageType schema; | ||
| private ByteBufferAllocator allocator; | ||
| private int columnIndexTruncateLength = ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH; | ||
| private boolean pageWriteChecksumEnabled = ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED; | ||
| private InternalFileEncryptor fileEncryptor = null; | ||
| private int rowGroupOrdinal = 0; | ||
|
|
||
| private Builder() {} | ||
|
|
||
| /** | ||
| * Use a single compressor for every column. | ||
| */ | ||
| public Builder withCompressor(BytesInputCompressor compressor) { | ||
| this.compressorProvider = path -> compressor; | ||
| return this; | ||
| } | ||
|
|
||
| /** | ||
| * Resolve the compressor per column from the given codec factory and properties, allowing | ||
| * per-column compression codecs. | ||
| */ | ||
| public Builder withCodecFactory(CompressionCodecFactory codecFactory, ParquetProperties props) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It looks a little bit weird that |
||
| this.compressorProvider = path -> codecFactory.getCompressor(props.getCompressionCodec(path)); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withSchema(MessageType schema) { | ||
| this.schema = schema; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withAllocator(ByteBufferAllocator allocator) { | ||
| this.allocator = allocator; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withColumnIndexTruncateLength(int columnIndexTruncateLength) { | ||
| this.columnIndexTruncateLength = columnIndexTruncateLength; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withPageWriteChecksumEnabled(boolean pageWriteChecksumEnabled) { | ||
| this.pageWriteChecksumEnabled = pageWriteChecksumEnabled; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withFileEncryptor(InternalFileEncryptor fileEncryptor) { | ||
| this.fileEncryptor = fileEncryptor; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder withRowGroupOrdinal(int rowGroupOrdinal) { | ||
| this.rowGroupOrdinal = rowGroupOrdinal; | ||
| return this; | ||
| } | ||
|
|
||
| public ColumnChunkPageWriteStore build() { | ||
| if (compressorProvider == null) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems that |
||
| throw new IllegalStateException("A compressor or codec factory must be set"); | ||
| } | ||
| return new ColumnChunkPageWriteStore( | ||
| compressorProvider, | ||
| schema, | ||
| allocator, | ||
| columnIndexTruncateLength, | ||
| pageWriteChecksumEnabled, | ||
| fileEncryptor, | ||
| rowGroupOrdinal); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public PageWriter getPageWriter(ColumnDescriptor path) { | ||
| return writers.get(path); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.