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

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_r432381728



##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java
##########
@@ -0,0 +1,146 @@
+/* 
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+import org.apache.parquet.hadoop.codec.ZstandardCodec;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.mapred.DeprecatedParquetOutputFormat;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Random;
+
+public class TestZstandardCodec {
+
+  private final Path inputPath = new 
Path("src/test/java/org/apache/parquet/hadoop/example/TestInputOutputFormat.java");
+
+  @Test
+  public void testZstdCodec() throws IOException {
+    ZstandardCodec codec = new ZstandardCodec();
+    Configuration conf = new Configuration();
+    int[] levels = {1, 4, 7, 10, 13, 16, 19, 22};
+    int[] dataSizes = {0, 1, 10, 1024, 1024 * 1024};
+
+    for (int i = 0; i < levels.length; i++) {
+      conf.setInt(ZstandardCodec.PARQUET_COMPRESS_ZSTD_LEVEL, levels[i]);
+      codec.setConf(conf);
+      for (int j = 0; j < dataSizes.length; j++) {
+        testZstd(codec, dataSizes[j]);
+      }
+    }
+  }
+
+  private void testZstd(ZstandardCodec codec, int dataSize) throws IOException 
{
+    byte[] data = new byte[dataSize];
+    (new Random()).nextBytes(data);
+    BytesInput compressedData = compress(codec,  BytesInput.from(data));
+    BytesInput decompressedData = decompress(codec, compressedData, 
data.length);
+    Assert.assertArrayEquals(data, decompressedData.toByteArray());
+  }
+
+  private BytesInput compress(ZstandardCodec codec, BytesInput bytes) throws 
IOException {
+    ByteArrayOutputStream compressedOutBuffer = new 
ByteArrayOutputStream((int)bytes.size());
+    CompressionOutputStream cos = 
codec.createOutputStream(compressedOutBuffer, null);
+    bytes.writeAllTo(cos);
+    cos.close();
+    return BytesInput.from(compressedOutBuffer);
+  }
+
+  private BytesInput decompress(ZstandardCodec codec, BytesInput bytes, int 
uncompressedSize) throws IOException {
+    BytesInput decompressed;
+    InputStream is = codec.createInputStream(bytes.toInputStream(), null);
+    decompressed = BytesInput.from(is, uncompressedSize);
+    is.close();
+    return decompressed;
+  }
+
+  @Test
+  public void testZstdConfWithMr() throws Exception {
+    JobConf jobConf = new JobConf();
+    Configuration conf = new Configuration();
+    jobConf.setInt(ZstandardCodec.PARQUET_COMPRESS_ZSTD_LEVEL, 18);
+    jobConf.setInt(ZstandardCodec.PARQUET_COMPRESS_ZSTD_LEVEL, 4);
+    RunningJob mapRedJob = runMapReduceJob(CompressionCodecName.ZSTD, jobConf, 
conf);
+    assert(mapRedJob.isSuccessful());
+  }
+
+  private RunningJob runMapReduceJob(CompressionCodecName codec, JobConf 
jobConf, Configuration conf) throws IOException, ClassNotFoundException, 
InterruptedException {

Review comment:
       Maybe, if we create two parquet files with different levels (e.g. 1 and 
22) but with the exact same data we can expect that the larger level will 
generate smaller file?
   I just want to ensure that the properties really arrive to the codec. If you 
think the related test requires too much effort or even not feasible I am fine 
if you can point me to the code path where the properties are passed through to 
the codecs.




----------------------------------------------------------------
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