[ 
https://issues.apache.org/jira/browse/PARQUET-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17113905#comment-17113905
 ] 

ASF GitHub Bot commented on PARQUET-1866:
-----------------------------------------

gszadovszky commented on a change in pull request #793:
URL: https://github.com/apache/parquet-mr/pull/793#discussion_r429112028



##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdCodec.java
##########
@@ -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 develop machine;

Review comment:
       ```suggestion
    * one because it requires 1) to set up hadoop on local development machine;
   ```

##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstdCodec.java
##########
@@ -0,0 +1,74 @@
+/* 
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.hadoop.codec.ZstdCodec;
+import org.junit.Assert;
+import org.junit.Test;  
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+
+public class TestZstdCodec {

Review comment:
       I tried to find the code part where we set the hadoop conf to the codec 
but could not find it. Please, write a high level test where you set 
compression level and workers in the hadoop conf and executes a file write via 
e.g. an MR job.

##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdCodec.java
##########
@@ -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 develop 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 ZstdCodec.
+ */
+public class ZstdCodec 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;

Review comment:
       Please, also update the documentation in the 
[README](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/README.md).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Replace Hadoop ZSTD with JNI-ZSTD
> ---------------------------------
>
>                 Key: PARQUET-1866
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1866
>             Project: Parquet
>          Issue Type: Improvement
>          Components: parquet-mr
>    Affects Versions: 1.12.0
>            Reporter: Xinli Shang
>            Assignee: Xinli Shang
>            Priority: Major
>             Fix For: 1.12.0
>
>
> The parquet-mr repo has been using 
> [ZSTD-JNI|https://github.com/luben/zstd-jni/tree/master/src/main/java/com/github/luben/zstd]
>  for the parquet-cli project. It is a cleaner approach to use this JNI than 
> using Hadoop ZSTD compression, because 1) on the developing box, installing 
> Hadoop is cumbersome, 2) Older version of Hadoop doesn't support ZSTD. 
> Upgrading Hadoop is another pain. This Jira is to replace Hadoop ZSTD with 
> ZSTD-JNI for parquet-hadoop project. 
> According to the author of ZSTD-JNI, Flink, Spark, Cassandra all use ZSTD-JNI 
> for ZSTD.
> Another approach is to use https://github.com/airlift/aircompressor which is 
> a pure Java implementation. But it seems the compression level is not 
> adjustable in aircompressor. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to