Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ public enum CompressionCodecName {
LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo"),
BROTLI("org.apache.hadoop.io.compress.BrotliCodec", CompressionCodec.BROTLI, ".br"),
LZ4("org.apache.hadoop.io.compress.Lz4Codec", CompressionCodec.LZ4, ".lz4"),
ZSTD("org.apache.hadoop.io.compress.ZStandardCodec", CompressionCodec.ZSTD, ".zstd");
ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", CompressionCodec.ZSTD, ".zstd");

public static CompressionCodecName fromConf(String name) {
if (name == null) {
Expand Down
13 changes: 12 additions & 1 deletion parquet-hadoop/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -324,9 +324,20 @@ ParquetInputFormat to materialize records. It should be a the descendant class o
**Property:** `parquet.read.schema`
**Description:** The read projection schema.


## Class: UnmaterializableRecordCounter

**Property:** `parquet.read.bad.record.threshold`
**Description:** The percentage of bad records to tolerate.
**Default value:** `0`

## Class: ZstandardCodec

**Property:** `parquet.compression.codec.zstd.level`
**Description:** The compression level of ZSTD. The valid range is 1~22. Generally the higher compression level, the higher compression ratio can be achieved, but the writing time will be longer.
**Default value:** `3`

---

**Property:** `parquet.compression.codec.zstd.workers`
**Description:** The number of threads will be spawned to compress in parallel. More workers improve speed, but also increase memory usage. When it is 0, it works as single-threaded mode.
**Default value:** `0`
5 changes: 5 additions & 0 deletions parquet-hadoop/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,11 @@
<version>${brotli-codec.version}</version>
<optional>true</optional>
</dependency>
<dependency>
<groupId>com.github.luben</groupId>
<artifactId>zstd-jni</artifactId>
<version>${zstd-jni.version}</version>
</dependency>

<dependency>
<groupId>com.google.guava</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,9 @@ class HeapBytesDecompressor extends BytesDecompressor {
public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
final BytesInput decompressed;
if (codec != null) {
decompressor.reset();
if (decompressor != null) {
decompressor.reset();
}
InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
decompressed = BytesInput.from(is, uncompressedSize);
} else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.codec;

import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionInputStream;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;

/**
* ZSTD compression codec for Parquet. We do not use the default hadoop
* one because it requires 1) to set up hadoop on local development machine;
* 2) to upgrade hadoop to the newer version to have ZSTD support which is
* more cumbersome than upgrading parquet version.
*
* This implementation relies on ZSTD JNI(https://github.com/luben/zstd-jni)
* which is already a dependency for Parquet. ZSTD JNI ZstdOutputStream and
* ZstdInputStream use Zstd internally. So no need to create compressor and
* decompressor in ZstandardCodec.
*/
public class ZstandardCodec implements Configurable, CompressionCodec {

public final static String PARQUET_COMPRESS_ZSTD_LEVEL = "parquet.compression.codec.zstd.level";
public final static int DEFAULT_PARQUET_COMPRESS_ZSTD_LEVEL = 3;
public final static String PARQUET_COMPRESS_ZSTD_WORKERS = "parquet.compression.codec.zstd.workers";
public final static int DEFAULTPARQUET_COMPRESS_ZSTD_WORKERS = 0;

private Configuration conf;

@Override
public void setConf(Configuration conf) {
this.conf = conf;
}

@Override
public Configuration getConf() {
return conf;
}

@Override
public Compressor createCompressor() {
// ZstdOutputStream calls static Zstd compressor, so no compressor is created
return null;
}

@Override
public Decompressor createDecompressor() {
// ZstdInputStream calls static Zstd decompressor, so no decompressor is created
return null;
}

@Override
public CompressionInputStream createInputStream(InputStream stream, Decompressor decompressor) throws IOException {
// Ignore decompressor because ZstdInputStream calls static Zstd decompressor
return createInputStream(stream);
}

@Override
public CompressionInputStream createInputStream(InputStream stream) throws IOException {
return new ZstdDecompressorStream(stream);
}

@Override
public CompressionOutputStream createOutputStream(OutputStream stream, Compressor compressor) throws IOException {
// Ignore compressor because ZstdOutputStream calls static Zstd compressor
return createOutputStream(stream);
}

@Override
public CompressionOutputStream createOutputStream(OutputStream stream) throws IOException {
return new ZstdCompressorStream(stream, conf.getInt(PARQUET_COMPRESS_ZSTD_LEVEL, DEFAULT_PARQUET_COMPRESS_ZSTD_LEVEL),
conf.getInt(PARQUET_COMPRESS_ZSTD_WORKERS, DEFAULTPARQUET_COMPRESS_ZSTD_WORKERS));
}

@Override
public Class<? extends Compressor> getCompressorType() {
return null;
}

@Override
public Class<? extends Decompressor> getDecompressorType() {
return null;
}

@Override
public String getDefaultExtension() {
return ".zstd";
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.codec;

import com.github.luben.zstd.ZstdOutputStream;
import org.apache.hadoop.io.compress.CompressionOutputStream;

import java.io.IOException;
import java.io.OutputStream;

public class ZstdCompressorStream extends CompressionOutputStream {

private ZstdOutputStream zstdOutputStream;

public ZstdCompressorStream(OutputStream stream, int level, int workers) throws IOException {
super(stream);
zstdOutputStream = new ZstdOutputStream(stream, level);
zstdOutputStream.setWorkers(workers);
}

public void write(byte[] b, int off, int len) throws IOException {
zstdOutputStream.write(b, off, len);
}

public void write(int b) throws IOException {
zstdOutputStream.write(b);
}

public void finish() throws IOException {
//no-opt, doesn't apply to ZSTD
}

public void resetState() throws IOException {
// no-opt, doesn't apply to ZSTD
}

@Override
public void flush() throws IOException {
zstdOutputStream.flush();
}

@Override
public void close() throws IOException {
zstdOutputStream.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.codec;

import com.github.luben.zstd.ZstdInputStream;
import org.apache.hadoop.io.compress.CompressionInputStream;

import java.io.IOException;
import java.io.InputStream;

public class ZstdDecompressorStream extends CompressionInputStream {

private ZstdInputStream zstdInputStream;

public ZstdDecompressorStream(InputStream stream) throws IOException {
super(stream);
zstdInputStream = new ZstdInputStream(stream);
}

public int read(byte[] b, int off, int len) throws IOException {
return zstdInputStream.read(b, off, len);
}

public int read() throws IOException {
return zstdInputStream.read();
}

public void resetState() throws IOException {
// no-opt, doesn't apply to ZSTD
}
}
Loading