Skip to content

Commit

Permalink
Add support for reading LZ4 and ZSTD when writing Hive tables
Browse files Browse the repository at this point in the history
Cherry-pick of trinodb/trino#910
Co-authored-by: Martin Traverso [email protected]

More details about ZSTD in Hive tables:

ZSTD is a more efficient compression mechanism, and in our production, we see ~7% reduction in storage size when converting from GZIP to ZSTD, and ~39% reduction from SNAPPY to ZSTD using default compression level (3).

Test Plan - Original commit in TrinoDB didn't have unit tests. Unit tests is added in this PR.
Tests of reading ZSTD files were done internally and also this change is used in production for a lot of tables.
  • Loading branch information
shangxinli authored and zhenxiao committed Nov 26, 2021
1 parent 71f3788 commit e17ad78
Show file tree
Hide file tree
Showing 2 changed files with 108 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,10 @@
package com.facebook.presto.parquet;

import io.airlift.compress.Decompressor;
import io.airlift.compress.lz4.Lz4Decompressor;
import io.airlift.compress.lzo.LzoDecompressor;
import io.airlift.compress.snappy.SnappyDecompressor;
import io.airlift.compress.zstd.ZstdDecompressor;
import io.airlift.slice.Slice;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;

Expand Down Expand Up @@ -56,6 +58,10 @@ public static Slice decompress(CompressionCodecName codec, Slice input, int unco
return input;
case LZO:
return decompressLZO(input, uncompressedSize);
case LZ4:
return decompressLz4(input, uncompressedSize);
case ZSTD:
return decompressZstd(input, uncompressedSize);
default:
throw new ParquetCorruptionException("Codec not supported in Parquet: " + codec);
}
Expand All @@ -68,6 +74,13 @@ private static Slice decompressSnappy(Slice input, int uncompressedSize)
return wrappedBuffer(buffer);
}

private static Slice decompressZstd(Slice input, int uncompressedSize)
{
byte[] buffer = new byte[uncompressedSize];
decompress(new ZstdDecompressor(), input, 0, input.length(), buffer, 0);
return wrappedBuffer(buffer);
}

private static Slice decompressGzip(Slice input, int uncompressedSize)
throws IOException
{
Expand Down Expand Up @@ -96,6 +109,13 @@ private static Slice decompressGzip(Slice input, int uncompressedSize)
return wrappedBuffer(buffer, 0, bytesRead);
}

private static Slice decompressLz4(Slice input, int uncompressedSize)
{
byte[] buffer = new byte[uncompressedSize];
decompress(new Lz4Decompressor(), input, 0, input.length(), buffer, 0);
return wrappedBuffer(buffer);
}

private static Slice decompressLZO(Slice input, int uncompressedSize)
{
LzoDecompressor lzoDecompressor = new LzoDecompressor();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* Licensed 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 com.facebook.presto.parquet;

import io.airlift.compress.Compressor;
import io.airlift.compress.lz4.Lz4Compressor;
import io.airlift.compress.zstd.ZstdCompressor;
import io.airlift.slice.Slice;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.testng.annotations.Test;

import java.io.IOException;

import static io.airlift.slice.Slices.wrappedBuffer;
import static org.apache.parquet.hadoop.metadata.CompressionCodecName.LZ4;
import static org.apache.parquet.hadoop.metadata.CompressionCodecName.ZSTD;
import static org.testng.Assert.assertEquals;

public class TestParquetCompressionUtils
{
@Test
public void testDecompressZSTD()
throws IOException
{
performTest(ZSTD, 0);
performTest(ZSTD, 1);
performTest(ZSTD, 100);
performTest(ZSTD, 256);
performTest(ZSTD, 512);
performTest(ZSTD, 1024);
}

@Test
public void testDecompressLZ4()
throws IOException
{
performTest(LZ4, 0);
performTest(LZ4, 1);
performTest(LZ4, 100);
performTest(LZ4, 256);
performTest(LZ4, 512);
performTest(LZ4, 1024);
}

private void performTest(CompressionCodecName codec, int inputLength)
throws IOException
{
Compressor compressor = null;
if (codec.equals(ZSTD)) {
compressor = new ZstdCompressor();
}
else if (codec.equals(LZ4)) {
compressor = new Lz4Compressor();
}

byte[] input = createArray(inputLength);
byte[] output = new byte[inputLength + 512];
int retLength = compress(compressor, input, inputLength, output, 0);

Slice decompressedSlice = ParquetCompressionUtils.decompress(codec, wrappedBuffer(output, 0, retLength), inputLength);
assertEquals(decompressedSlice, wrappedBuffer(input));
}

private byte[] createArray(int length)
{
byte[] data = new byte[length];
for (int i = 0; i < length; i++) {
data[i] = (byte) (i % 256);
}
return data;
}

private static int compress(Compressor compressor, byte[] byteArray, int inputLength, byte[] output, int outputOffset)
{
return compressor.compress(byteArray, 0, inputLength, output, outputOffset, output.length - outputOffset);
}
}

0 comments on commit e17ad78

Please sign in to comment.