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

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

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


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLz4RawCodec.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.parquet.hadoop.codec.*;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void TestLz4Raw() throws IOException {
+    // Reuse the snappy objects between test cases

Review Comment:
   "snappy"?



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLz4RawCodec.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.parquet.hadoop.codec.*;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void TestLz4Raw() throws IOException {
+    // Reuse the snappy objects between test cases
+    Lz4RawCompressor compressor = new Lz4RawCompressor();
+    Lz4RawDecompressor decompressor = new Lz4RawDecompressor();
+
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "FooBar1", "FooBar2");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "a", "blahblahblah", "abcdef");
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+  }
+
+  private void TestLz4Raw(Lz4RawCompressor compressor, Lz4RawDecompressor 
decompressor,
+                          String... strings) throws IOException {
+    compressor.reset();
+    decompressor.reset();
+
+    int uncompressedSize = 0;
+    for (String s : strings) {
+      uncompressedSize += s.length();
+    }
+    byte[] uncompressedData = new byte[uncompressedSize];
+    int len = 0;
+    for (String s : strings) {
+      byte[] tmp = s.getBytes();
+      System.arraycopy(tmp, 0, uncompressedData, len, s.length());
+      len += s.length();
+    }

Review Comment:
   Also what's the point of passing several strings if you're just 
concatenating them together?



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLz4RawCodec.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.parquet.hadoop.codec.*;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void TestLz4Raw() throws IOException {
+    // Reuse the snappy objects between test cases
+    Lz4RawCompressor compressor = new Lz4RawCompressor();
+    Lz4RawDecompressor decompressor = new Lz4RawDecompressor();
+
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "FooBar1", "FooBar2");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "a", "blahblahblah", "abcdef");
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+  }
+
+  private void TestLz4Raw(Lz4RawCompressor compressor, Lz4RawDecompressor 
decompressor,
+                          String... strings) throws IOException {
+    compressor.reset();
+    decompressor.reset();
+
+    int uncompressedSize = 0;
+    for (String s : strings) {
+      uncompressedSize += s.length();
+    }
+    byte[] uncompressedData = new byte[uncompressedSize];
+    int len = 0;
+    for (String s : strings) {
+      byte[] tmp = s.getBytes();
+      System.arraycopy(tmp, 0, uncompressedData, len, s.length());
+      len += s.length();
+    }
+
+    assert (compressor.needsInput());
+    compressor.setInput(uncompressedData, 0, len);
+    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);

Review Comment:
   Can you also check the compressor and decompressor work ok in a streaming 
fashion?
   



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLz4RawCodec.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.parquet.hadoop.codec.*;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void TestLz4Raw() throws IOException {
+    // Reuse the snappy objects between test cases
+    Lz4RawCompressor compressor = new Lz4RawCompressor();
+    Lz4RawDecompressor decompressor = new Lz4RawDecompressor();
+
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "FooBar1", "FooBar2");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "a", "blahblahblah", "abcdef");
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+  }

Review Comment:
   Would be nice to also test with larger compressible payloads.



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLz4RawCodec.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.parquet.hadoop.codec.*;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestLz4RawCodec {
+  @Test
+  public void TestLz4Raw() throws IOException {
+    // Reuse the snappy objects between test cases
+    Lz4RawCompressor compressor = new Lz4RawCompressor();
+    Lz4RawDecompressor decompressor = new Lz4RawDecompressor();
+
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "FooBar1", "FooBar2");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+    TestLz4Raw(compressor, decompressor, "a", "blahblahblah", "abcdef");
+    TestLz4Raw(compressor, decompressor, "");
+    TestLz4Raw(compressor, decompressor, "FooBar");
+  }
+
+  private void TestLz4Raw(Lz4RawCompressor compressor, Lz4RawDecompressor 
decompressor,
+                          String... strings) throws IOException {
+    compressor.reset();
+    decompressor.reset();
+
+    int uncompressedSize = 0;
+    for (String s : strings) {
+      uncompressedSize += s.length();
+    }
+    byte[] uncompressedData = new byte[uncompressedSize];
+    int len = 0;
+    for (String s : strings) {
+      byte[] tmp = s.getBytes();
+      System.arraycopy(tmp, 0, uncompressedData, len, s.length());
+      len += s.length();
+    }

Review Comment:
   
   
   Java doesn't java an easier way to do this??
   





> 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