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

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

pitrou commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r997972294


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestLz4RawCodec.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.Configuration;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.hadoop.codec.*;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void testBlock() throws IOException {
+    // Reuse the lz4 objects between test cases
+    Lz4RawCompressor compressor = new Lz4RawCompressor();
+    Lz4RawDecompressor decompressor = new Lz4RawDecompressor();
+
+    testBlockCompression(compressor, decompressor, "");
+    testBlockCompression(compressor, decompressor, "FooBar");
+    testBlockCompression(compressor, decompressor, "FooBar1FooBar2");
+    testBlockCompression(compressor, decompressor, "FooBar");
+    testBlockCompression(compressor, decompressor, "ablahblahblahabcdef");
+    testBlockCompression(compressor, decompressor, "");
+    testBlockCompression(compressor, decompressor, "FooBar");
+  }
+
+  // Test lz4 raw compression in the block fashion
+  private void testBlockCompression(Lz4RawCompressor compressor, 
Lz4RawDecompressor decompressor,
+                                    String data) throws IOException {
+    compressor.reset();
+    decompressor.reset();
+
+    int uncompressedSize = data.length();
+    byte[] uncompressedData = data.getBytes();
+
+    assert (compressor.needsInput());
+    compressor.setInput(uncompressedData, 0, uncompressedSize);
+    assert (compressor.needsInput());
+    compressor.finish();
+    assert (!compressor.needsInput());
+    assert (!compressor.finished() || uncompressedSize == 0);
+    byte[] compressedData = new byte[1000];
+
+    int compressedSize = compressor.compress(compressedData, 0, 1000);
+    assert (compressor.finished());
+
+    assert (!decompressor.finished());
+    assert (decompressor.needsInput());
+    decompressor.setInput(compressedData, 0, compressedSize);
+    assert (!decompressor.finished());
+    byte[] decompressedData = new byte[uncompressedSize];
+    int decompressedSize = decompressor.decompress(decompressedData, 0, 
uncompressedSize);
+    assert (decompressor.finished());
+
+    assertEquals(uncompressedSize, decompressedSize);
+    assertArrayEquals(uncompressedData, decompressedData);
+  }
+
+  // Test lz4 raw compression in the streaming fashion
+  @Test
+  public void testCodec() throws IOException {
+    Lz4RawCodec codec = new Lz4RawCodec();
+    Configuration conf = new Configuration();
+    int[] bufferSizes = {128, 1024, 4 * 1024, 16 * 1024, 128 * 1024, 1024 * 
1024};
+    int[] dataSizes = {0, 1, 10, 1024, 1024 * 1024};
+
+    for (int i = 0; i < bufferSizes.length; i++) {
+      conf.setInt(Lz4RawCodec.BUFFER_SIZE_CONFIG, bufferSizes[i]);
+      codec.setConf(conf);
+      for (int j = 0; j < dataSizes.length; j++) {
+        testLz4RawCodec(codec, dataSizes[j]);
+      }
+    }
+  }
+
+  private void testLz4RawCodec(Lz4RawCodec codec, int dataSize) throws 
IOException {
+    byte[] data = new byte[dataSize];
+    (new Random()).nextBytes(data);

Review Comment:
   This produces basically uncompressible data? That seems a bit insufficient.
   To make the data more compressible, you can for example repeat the same 
chunk N times.





> Support LZ4_RAW codec
> ---------------------
>
>                 Key: PARQUET-2196
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2196
>             Project: Parquet
>          Issue Type: Improvement
>          Components: parquet-mr
>            Reporter: Gang Wu
>            Priority: Major
>
> There is a long history about the LZ4 interoperability of parquet files 
> between parquet-mr and parquet-cpp (which is now in the Apache Arrow). 
> Attached links are the evidence. In short, a new LZ4_RAW codec type has been 
> introduced since parquet format v2.9.0. However, only parquet-cpp supports 
> LZ4_RAW. The parquet-mr library still uses the old Hadoop-provided LZ4 codec 
> and cannot read parquet files with LZ4_RAW.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to