[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-11-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli merged PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-11-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1300583034

   LGTM, let's see if there are comments from others, otherwise we can merge.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4raw");

Review Comment:
   Sounds good to me.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   > Ok, but can you reuse the more explanatory comment from the other file?
   > 
   > ```
   > // SnappyUtil was dedicated to SnappyCodec in the past. Now it is used 
for both
   > // NonBlockedDecompressor and NonBlockedCompressor without renaming 
due to its
   > // dependency by some external downstream projects.
   > ```
   
   Done





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4raw");

Review Comment:
   I searched the parquet-mr repo and found the only usage of the codec 
extension as below: 
   
   
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L420
   ```java
 public RecordWriter getRecordWriter(TaskAttemptContext 
taskAttemptContext, Mode mode)
 throws IOException, InterruptedException {
   
   final Configuration conf = getConfiguration(taskAttemptContext);
   
   CompressionCodecName codec = getCodec(taskAttemptContext);
   String extension = codec.getExtension() + ".parquet";
   Path file = getDefaultWorkFile(taskAttemptContext, extension);
   return getRecordWriter(conf, file, codec, mode);
 }
   ```
   
   The extension is part of the enum definition and cannot be removed. I 
propose to rename `lz4` to `lz4hadoop` to explicitly differentiate this lz4 
variant from standard `lz4`.  @pitrou 





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

pitrou commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1290291241

   @lwhite1 Could you perhaps review this? I can't really comment on the 
details of the Java code, myself.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   Ok, but can you reuse the more explanatory comment from the other file?
   ```
   // SnappyUtil was dedicated to SnappyCodec in the past. Now it is used 
for both
   // NonBlockedDecompressor and NonBlockedCompressor without renaming due 
to its
   // dependency by some external downstream projects.
   ```





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   Ok, but can the comment be more explanatory? Something like:
   ```
   // The `SnappyUtil` name has to be kept as the API has been used
   // by downstream projects, even though we also use it for non-Snappy
   // codecs.
   ```





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4raw");

Review Comment:
   I think both file extensions for LZ4 and LZ4RAW should be removed.
   The reason is that the `lz4` command-line utilities expect the LZ4 frame 
format.
   In any case, the `.lz4` extension should certainly not be used for the 
"Hadoop LZ4" compression codec.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1288388631

   I have changed the approach of interop test to follow encryption which 
downloads test files from parquet-testing repo and verifies the data 
decompressed and decoded from them. 
   
   In addition, the codec test has been modified to be generic which supports 
LZ4_RAW and SNAPPY. It should be easy to support more codec types.
   
   Please take a look when you have time and let me know if there is any 
feedback, thanks! @pitrou @emkornfield @ggershinsky  @shangxinli 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


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

Review Comment:
   > You could at least write this one as a generic test, specialize it for all 
existing codecs, while also keeping the existing dedicated tests to ensure we 
don't lose coverage. Does that sound reasonable?
   
   I agree. I have changed the test to be generic to use only CompressionCodec, 
Compressor and Decompressor in my latest patch. For now I only enable LZ4 RAW 
in the test due to missing JNI library on my Mac for verification. Should be 
easy to extend for other codecs. Please take a look, thanks! @pitrou 





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


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

Review Comment:
   You could at least write this one as a generic test, specialize it for all 
existing codecs, while also keeping the existing dedicated tests to ensure we 
don't lose coverage. Does that sound reasonable?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.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;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0. It differs from the Lz4Codec
+ * shipped with Apache Hadoop by removing the light frame header which includes
+ * 4 byte for uncompressed length and 4 byte for compressed length. In the
+ * Apache Arrow implementation, these two Lz4 codecs are recognized as LZ4_RAW
+ * and LZ4_HADOOP to minimize the confusion at its best. Please check the link
+ * below for reference.
+ * https://github.com/apache/parquet-format/blob/master/Compression.md
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+
+  // Hadoop config for how big to make intermediate buffers.
+  public static final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  private static final int DEFAULT_BUFFER_SIZE_CONFIG = 4 * 1024;

Review Comment:
   Fair enough. The value seems much too small to me, but let's keep it like 
this anyway.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


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

Review Comment:
   Well, writing a dedicated test is non-trivial either :-) And the more you 
pile up dedicated tests, the less trivial it will be to reconcile them...





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
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:
   Added cases to repeat chunks.





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

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.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;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0. It differs from the Lz4Codec
+ * shipped with Apache Hadoop by removing the light frame header which includes
+ * 4 byte for uncompressed length and 4 byte for compressed length. In the
+ * Apache Arrow implementation, these two Lz4 codecs are recognized as LZ4_RAW
+ * and LZ4_HADOOP to minimize the confusion at its best. Please check the link
+ * below for reference.
+ * https://github.com/apache/parquet-format/blob/master/Compression.md
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+
+  // Hadoop config for how big to make intermediate buffers.
+  public static final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  private static final int DEFAULT_BUFFER_SIZE_CONFIG = 4 * 1024;

Review Comment:
   In order to keep the same behavior with the snappy codec, I think it is 
better to keep the same default value.



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset 

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,71 @@
+/*
+ * 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.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  private static String SIMPLE_FILE = "/lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "/lz4_raw_compressed_larger.parquet";
+
+  @Test
+  public void testInteropReadSimpleLz4RawParquetFile() throws IOException {
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = createPathFromCP(SIMPLE_FILE);
+readParquetFile(simpleFile, 4);
+  }
+
+  @Test
+  public void testInteropReadLargerLz4RawParquetFile() throws IOException {
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = createPathFromCP(LARGER_FILE);
+readParquetFile(largerFile, 1);
+  }
+
+  private static Path createPathFromCP(String path) {
+try {
+  return new 
Path(TestInteropReadLz4RawCodec.class.getResource(path).toURI());
+} catch (URISyntaxException e) {
+  throw new RuntimeException(e);
+}
+  }
+
+  private void readParquetFile(Path filePath, int expectedNumRows) throws 
IOException {
+try (ParquetReader reader = ParquetReader.builder(new 
GroupReadSupport(), filePath).build()) {
+  int numRows = 0;
+  while (reader.read() != null) {
+numRows++;
+  }
+  reader.close();
+  assertEquals(expectedNumRows, numRows);

Review Comment:
   I have follow the same approach of Parquet C++ to verify the data.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


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

Review Comment:
   That would be ideal. However, the test folder already has dedicated test 
cases for different codec, e.g. TestSnappyCodec and TestZstandardCodec. They do 
not use common or abstract class in the implementation so it is non-trivial to 
make a generic test for all codecs at the moment.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-22 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.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;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0. It differs from the Lz4Codec
+ * shipped with Apache Hadoop by removing the light frame header which includes
+ * 4 byte for uncompressed length and 4 byte for compressed length. In the
+ * Apache Arrow implementation, these two Lz4 codecs are recognized as LZ4_RAW
+ * and LZ4_HADOOP to minimize the confusion at its best. Please check the link
+ * below for reference.
+ * https://github.com/apache/parquet-format/blob/master/Compression.md
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+
+  // Hadoop config for how big to make intermediate buffers.
+  public static final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  private static final int DEFAULT_BUFFER_SIZE_CONFIG = 4 * 1024;

Review Comment:
   This is the buffer size used internally by 
org.apache.hadoop.io.compress.CompressorStream and 
org.apache.hadoop.io.compress.DecompressorStream. I copied the same default 
value from SnappyCodec: 
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyCodec.java#L75





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-19 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

ggershinsky commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1283562000

   > > Yep, we test encryption interop using binary files in the 
parquet-testing repo. @wgtmac Please have a look at this code: 
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestEncryptionOptions.java#L130
   > 
   > @ggershinsky @emkornfield Yes I did that in the previous commit: 
[6e02cb2#diff-61de8043597197e5da4b42ebfe2d123b9c42e56fcb520e293c1a10ba55fa93cc](https://github.com/apache/parquet-mr/commit/6e02cb20eb580e80a793b02d187237770868b335#diff-61de8043597197e5da4b42ebfe2d123b9c42e56fcb520e293c1a10ba55fa93cc)
 but switched to local file to avoid unstable test.
   > 
   
   What was the source of the instability? 
   (encryption interop runs with files from the `parquet-testing` module for ~3 
years now, no problems as far as I remember)
   
   




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   I see. Can you add a comment about that?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


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

Review Comment:
   I'm curious, why are you creating dedicated test cases for LZ4 instead of 
sharing the same basic tests for all compressors/decompressors?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   This piece of code was originally used for SnappyCodec only. I have 
abstracted these to be the common part of Lz4RawCodec and SnappyCodec. I have 
tried to rename SnappyUtil but unfortunately this public API has been used by 
some downstream projects. That's why I decided to keep the original class name.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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: 

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,71 @@
+/*
+ * 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.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  private static String SIMPLE_FILE = "/lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "/lz4_raw_compressed_larger.parquet";
+
+  @Test
+  public void testInteropReadSimpleLz4RawParquetFile() throws IOException {
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = createPathFromCP(SIMPLE_FILE);
+readParquetFile(simpleFile, 4);
+  }
+
+  @Test
+  public void testInteropReadLargerLz4RawParquetFile() throws IOException {
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = createPathFromCP(LARGER_FILE);
+readParquetFile(largerFile, 1);
+  }
+
+  private static Path createPathFromCP(String path) {
+try {
+  return new 
Path(TestInteropReadLz4RawCodec.class.getResource(path).toURI());
+} catch (URISyntaxException e) {
+  throw new RuntimeException(e);
+}
+  }
+
+  private void readParquetFile(Path filePath, int expectedNumRows) throws 
IOException {
+try (ParquetReader reader = ParquetReader.builder(new 
GroupReadSupport(), filePath).build()) {
+  int numRows = 0;
+  while (reader.read() != null) {
+numRows++;
+  }
+  reader.close();
+  assertEquals(expectedNumRows, numRows);

Review Comment:
   Can you also test the actual data, at least part of it?
   Here's what Parquet C++ does: 
https://github.com/apache/arrow/blob/cd33544533ee7d70cd8ff7556e59ef8f1d33a176/cpp/src/parquet/reader_test.cc#L936-L1007





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedDecompressor.java:
##
@@ -0,0 +1,174 @@
+/*
+ * 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.io.compress.Decompressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+abstract public class NonBlockedDecompressor implements Decompressor {
+
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException if reading or decompression fails
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   Same question here.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedCompressor.java:
##
@@ -0,0 +1,186 @@
+/*
+ * 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.Compressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class is a wrapper around the underlying compressor. It always consumes
+ * the entire input in setInput and compresses it as one compressed block.
+ */
+abstract public class NonBlockedCompressor implements Compressor {
+
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   I'm a curious: why does this code call a Snappy-specific API?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.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;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0. It differs from the Lz4Codec
+ * shipped with Apache Hadoop by removing the light frame header which includes
+ * 4 byte for uncompressed length and 4 byte for compressed length. In the
+ * Apache Arrow implementation, these two Lz4 codecs are recognized as LZ4_RAW
+ * and LZ4_HADOOP to minimize the confusion at its best. Please check the link
+ * below for reference.
+ * https://github.com/apache/parquet-format/blob/master/Compression.md
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+
+  // Hadoop config for how big to make intermediate buffers.
+  public static final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  private static final int DEFAULT_BUFFER_SIZE_CONFIG = 4 * 1024;

Review Comment:
   Can you explain what this is used for?
   Also, 4 kiB seems extremely small for compression purposes.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

pitrou commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1282107004

   It would probably be easier to use a git submodule (I wonder why that's not 
the approach being used in `parquet-mr`), but in any case reference files 
should go into the `parquet-testing` repo, not be privately-maintained in 
`parquet-mr`.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1282105001

   > Yep, we test encryption interop using binary files in the parquet-testing 
repo. @wgtmac Please have a look at this code: 
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestEncryptionOptions.java#L130
   
   @ggershinsky @emkornfield  Yes I did that in the previous commit: 
https://github.com/apache/parquet-mr/pull/1000/commits/6e02cb20eb580e80a793b02d187237770868b335#diff-61de8043597197e5da4b42ebfe2d123b9c42e56fcb520e293c1a10ba55fa93cc
 but switched to local file to avoid unstable test.
   
   @shangxinli Should I keep the resource file approach or switch to the 
download approach?




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

ggershinsky commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1281945116

   Yep, we test encryption interop using binary files in the parquet-testing 
repo. @wgtmac Please have a look at this code: 
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestEncryptionOptions.java#L130




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

emkornfield commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1281847541

   FWIW, Arrow/Parquet C++ checkout https://github.com/apache/parquet-testing 
when running parquet tests (instead of checking binary files into the main 
repo).
   
   As an aside, I think we do need better interop testing between parquet 
implementations, it wouldn't be trivial, but it would be really nice to have 
compatibility tests.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1281720715

   Hm... any opinion on this @ggershinsky ? 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1281706700

   > Looks good. The only thing is we checked in binary files directly. It 
would be hard to maintain in the future. Can you generate the parquet file 
using the parquetwriter?
   
   @shangxinli  The parquet file was created by the parquet C++ writer from 
Apache Arrow which aims to test the interoperability. I can generate the 
parquet file using the java writer but it no longer meets the requirement.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1281244709

   Looks good. The only thing is we checked in binary files directly. It would 
be hard to maintain in the future. Can you generate the parquet file using the 
parquetwriter? 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   @shangxinli  I have checked that **io.airlift:aircompressor** does not bring 
any new dependency. Please check the following result:
   
   > ✗ mvn dependency:tree
   > [INFO] Scanning for projects...
   > [INFO]
   > [INFO] -< org.apache.parquet:parquet-hadoop 
>--
   > [INFO] Building Apache Parquet Hadoop 1.13.0-SNAPSHOT
   > [INFO] [ jar 
]-
   > [INFO]
   > [INFO] -

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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   From the 
[pom.xml](https://github.com/airlift/aircompressor/blob/master/pom.xml), its 
dependencies are either **_provided_** or **_test_**.
   
   





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   @shangxinli  I have checked that **io.airlift:aircompressor** does not bring 
any new dependency. Please check the following result:
   
   > ✗ mvn dependency:tree
   > [INFO] Scanning for projects...
   > [INFO]
   > [INFO] -< org.apache.parquet:parquet-hadoop 
>--
   > [INFO] Building Apache Parquet Hadoop 1.13.0-SNAPSHOT
   > [INFO] [ jar 
]-
   > [INFO]
   > [INFO] -

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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * 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 okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();

Review Comment:
   > That would be the last option. Too much downloading will make the build 
unreliable.
   
   I have changed the interop test to read files from resource.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,103 @@
+/*
+ * 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.*;

Review Comment:
   Fixed





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   Change the suffix to ".lz4raw"





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   Change the short name to "F".





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawDecompressor.java:
##
@@ -0,0 +1,46 @@
+/*
+ * 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 io.airlift.compress.lz4.Lz4Decompressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawDecompressor extends NonBlockedDecompressor {
+
+  private Lz4Decompressor decompressor = new Lz4Decompressor();
+
+  @Override
+  protected int uncompress(ByteBuffer compressed, ByteBuffer uncompressed) 
throws IOException {
+decompressor.decompress(compressed, uncompressed);
+int uncompressedSize = uncompressed.position();
+uncompressed.limit(uncompressedSize);
+uncompressed.rewind();
+return uncompressedSize;
+  }
+
+  @Override
+  protected int uncompressedLength(ByteBuffer compressed, int 
maxUncompressedLength) throws IOException {

Review Comment:
   Renamed to maxUncompressedLength





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * 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 okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();

Review Comment:
   That would be the last option. Too much downloading will make the build 
unreliable. 





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   Generally, we are strict to add more dependencies. Do we know if this 
dependency also brings in other dependencies too? What total libraries were 
introduced and what is their footprint of them? 





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-10-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,103 @@
+/*
+ * 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.*;

Review Comment:
   Please don't use wildcard





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   We don't register file extensions in C++.
   That said, ".lz4" probably can't be appropriate for the Hadoop LZ4 codec.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   In fact they are the same compression except that the LZ4_HADOOP adds extra 
8 bytes for uncompressed size and compressed size. Check here for reference: 
https://github.com/airlift/aircompressor/blob/master/src/main/java/io/airlift/compress/lz4/HadoopLz4OutputStream.java#L97



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();

Review Comment:
   I don't think this is an issue. parquet-mr creates CompressionInputStream 
and CompressionOutputStream based on new allocated Compressor and Decompressor 
objects.  ZstandardCodec even returns null for the createCompressor and 
createDecompressor methods.



##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   I haven't found the suffix parity in the C++ implementation. But I can 
switch it to "lz4_raw" if that makes sense: 
https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/compression.cc#L51



##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * 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 okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a 

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * 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 okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();

Review Comment:
   is it common in parquet MR to download files automatically as part of tests?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * 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 okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();
+  Response response = httpClient.newCall(request).execute();
+  if (!response.isSuccessful()) {
+throw new IOException("Failed to download file: " + response);
+  }
+  try (FSDataOutputStream fdos = fs.create(file)) {
+fdos.write(response.body().bytes());
+  }
+}
+return file;
+  }
+
+  private void readParquetFile(Path filePath, int expectedNumRows) throws 
IOException {
+try (ParquetReader reader = ParquetReader.builder(new 
GroupReadSupport(), filePath).build()) {
+  int numRows = 0;
+  while (reader.read() != null) {
+numRows++;

Review Comment:
   should there be a deeper equality check here?





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

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();

Review Comment:
   is thread-safety and issue for 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)


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.*;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0.

Review Comment:
   it would be good to link to the spec or provide more details on how this 
differs from LZ4 non-raw.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   somebody more familiar with parquet-mr, on whether adding tis dependency is 
OK.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ 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.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   should there be a different extension?  I forget what we did in the C++ 
implemetation.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   this looks like the same value as LZ4, which doesn't seem correct, as they 
are in fact two different compression schemes? At least could we add a comment 
on explaining why they are the same?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   this looks like the same value as LZ4, which doesn't seem correct, as they 
are in fact two different compression schemes?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1262462916

   The interop test has been added. Please take a look again. Thanks! 
@shangxinli @pitrou 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1261165283

   In terms of the interop test, I plan to simply run the new ScanCommand 
(introduced by https://github.com/apache/parquet-mr/pull/998) of parquet-cli on 
the two lz4_raw parquet files in the parquet-testing directory. Will do it once 
the PR has been merged. WDYT? @pitrou @shangxinli 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
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:
   Fixed





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1261133491

   @shangxinli Thanks for the review. 
   
   I haven't be able to run the benchmark since the Hadoop Lz4Codec is 
implemented via JNI and I cannot simply run it on my Mac M1 laptop.
   
   Generally this is the same LZ4 algorithm with the one shipped by Hadoop. The 
key difference is that LZ4_RAW has removed the redundant block header 
introduced by Hadoop Lz4Codec and its implementation is in pure Java.
   
   I will add the interop test in a follow-up commit.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,99 @@
+/*
+ * 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.*;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0.
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+  // Hadoop config for how big to make intermediate buffers.
+  private final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  @Override
+  public void setConf(Configuration conf) {
+this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+return conf;
+  }
+
+  @Override
+  public Compressor createCompressor() {
+return new Lz4RawCompressor();
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+return new Lz4RawDecompressor();
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream)
+throws IOException {
+return createInputStream(stream, createDecompressor());
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream,
+  Decompressor decompressor) 
throws IOException {
+return new NonBlockedDecompressorStream(stream, decompressor,
+  conf.getInt(BUFFER_SIZE_CONFIG, 4*1024));

Review Comment:
   Fixed



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawDecompressor.java:
##
@@ -0,0 +1,46 @@
+/*
+ * 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 io.airlift.compress.lz4.Lz4Decompressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawDecompressor extends NonBlockedDecompressor {
+
+  private Lz4Decompressor decompressor = new Lz4Decompressor();
+
+  @Override
+  protected int uncompress(ByteBuffer compressed, ByteBuffer uncompressed) 
throws IOException {

Review Comment:
   Fixed





> 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] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedDecompressor.java:
##
@@ -0,0 +1,174 @@
+/*
+ * 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.io.compress.Decompressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+abstract public class NonBlockedDecompressor implements Decompressor {
+
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException if reading or decompression fails
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);
+if (inputBuffer.position() == 0 && !outputBuffer.hasRemaining()) {
+  return 0;
+}
+
+if (!outputBuffer.hasRemaining()) {
+  inputBuffer.rewind();
+  Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid 
position of 0.");
+  Preconditions.checkArgument(outputBuffer.position() == 0, "Invalid 
position of 0.");
+  // There is compressed input, decompress it now.
+  int decompressedSize = uncompressedLength(inputBuffer, len);
+  if (decompressedSize > outputBuffer.capacity()) {
+ByteBuffer oldBuffer = outputBuffer;
+outputBuffer = ByteBuffer.allocateDirect(decompressedSize);
+CleanUtil.cleanDirectBuffer(oldBuffer);
+  }
+
+  // Reset the previous outputBuffer (i.e. set position to 0)
+  outputBuffer.clear();
+  int size = uncompress(inputBuffer, outputBuffer);
+  outputBuffer.limit(size);
+  // We've decompressed the entire input, reset the input now
+  inputBuffer.clear();
+  inputBuffer.limit(0);
+  finished = true;
+}
+
+// Return compressed output up to 'len'
+int numBytes = Math.min(len, outputBuffer.remaining());
+outputBuffer.get(buffer, off, numBytes);
+return numBytes;
+  }
+
+  /**
+   * Sets input data for decompression.
+   * This should be called if and only if {@link #needsInput()} returns
+   * true indicating that more input data is required.
+   * (Both native and non-native versions of various Decompressors require
+   * that the data passed in via b[] remain unmodified until
+   * the caller is explicitly notified--via {@link #needsInput()}--that the
+   * buffer may be safely modified.  With this requirement, an extra
+   * buffer-copy can be avoided.)
+   *
+   * @param buffer Input data
+   * @param offStart offset
+   * @param lenLength
+   */
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {
+SnappyUtil.validateBuffer(buffer, off, len);
+
+if (inputBuffer.capacity() - inputBuffer.position() < len) {
+  final ByteBuffer newBuffer = 
ByteBuffer.allocateDirect(inputBuffer.position() + len);
+  inputBuffer.rewind();
+  newBuffer.put(inputBuffer);
+  final ByteBuffer oldBuffer = inputBuffer;
+  inputBuffer = newBuffer;
+  CleanUtil.cleanDirectBuffer(oldBuffer);
+} else {
+  inputBuffer.limit(inputBuffer.position() + len);
+}
+

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
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:
   I just copied from TestSnappyCodec. Fixed it with simpler implementation.



##
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) {
+  

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
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:
   Done





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedDecompressor.java:
##
@@ -0,0 +1,174 @@
+/*
+ * 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.io.compress.Decompressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+abstract public class NonBlockedDecompressor implements Decompressor {
+
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException if reading or decompression fails
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);
+if (inputBuffer.position() == 0 && !outputBuffer.hasRemaining()) {
+  return 0;
+}
+
+if (!outputBuffer.hasRemaining()) {
+  inputBuffer.rewind();
+  Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid 
position of 0.");
+  Preconditions.checkArgument(outputBuffer.position() == 0, "Invalid 
position of 0.");
+  // There is compressed input, decompress it now.
+  int decompressedSize = uncompressedLength(inputBuffer, len);
+  if (decompressedSize > outputBuffer.capacity()) {
+ByteBuffer oldBuffer = outputBuffer;
+outputBuffer = ByteBuffer.allocateDirect(decompressedSize);
+CleanUtil.cleanDirectBuffer(oldBuffer);
+  }
+
+  // Reset the previous outputBuffer (i.e. set position to 0)
+  outputBuffer.clear();
+  int size = uncompress(inputBuffer, outputBuffer);
+  outputBuffer.limit(size);
+  // We've decompressed the entire input, reset the input now
+  inputBuffer.clear();
+  inputBuffer.limit(0);
+  finished = true;
+}
+
+// Return compressed output up to 'len'
+int numBytes = Math.min(len, outputBuffer.remaining());
+outputBuffer.get(buffer, off, numBytes);
+return numBytes;
+  }
+
+  /**
+   * Sets input data for decompression.
+   * This should be called if and only if {@link #needsInput()} returns
+   * true indicating that more input data is required.
+   * (Both native and non-native versions of various Decompressors require
+   * that the data passed in via b[] remain unmodified until
+   * the caller is explicitly notified--via {@link #needsInput()}--that the
+   * buffer may be safely modified.  With this requirement, an extra
+   * buffer-copy can be avoided.)
+   *
+   * @param buffer Input data
+   * @param offStart offset
+   * @param lenLength
+   */
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   I have tried that, but the CI checks fail. I guess there are some external 
dependencies on SnappyUtil class already. So I decide not to touch it.





> Support LZ4_RAW codec
> -
>
> Key: PARQUET-2196
> URL: https://issues.apache.org/jira/browse/PARQUET-2196
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawDecompressor.java:
##
@@ -0,0 +1,46 @@
+/*
+ * 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 io.airlift.compress.lz4.Lz4Decompressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawDecompressor extends NonBlockedDecompressor {
+
+  private Lz4Decompressor decompressor = new Lz4Decompressor();
+
+  @Override
+  protected int uncompress(ByteBuffer compressed, ByteBuffer uncompressed) 
throws IOException {
+decompressor.decompress(compressed, uncompressed);
+int uncompressedSize = uncompressed.position();
+uncompressed.limit(uncompressedSize);
+uncompressed.rewind();
+return uncompressedSize;
+  }
+
+  @Override
+  protected int uncompressedLength(ByteBuffer compressed, int 
maxUncompressedLength) throws IOException {

Review Comment:
   I can rename to it. The purpose of this function is to return its 
uncompressed length at its best effort. If that is unavailable, try to return 
the upper bound of the uncompressed length.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();
+
+  @Override
+  protected int maxCompressedLength(int byteSize) {
+return 
io.airlift.compress.lz4.Lz4RawCompressor.maxCompressedLength(byteSize);

Review Comment:
   I have to use the full class name here since Java does not support importing 
type alias for classes with same name.





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1259976087

   Nice implementation! For the test, can you add more for interop with lz4?




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedDecompressor.java:
##
@@ -0,0 +1,174 @@
+/*
+ * 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.io.compress.Decompressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+abstract public class NonBlockedDecompressor implements Decompressor {
+
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException if reading or decompression fails
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);
+if (inputBuffer.position() == 0 && !outputBuffer.hasRemaining()) {
+  return 0;
+}
+
+if (!outputBuffer.hasRemaining()) {
+  inputBuffer.rewind();
+  Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid 
position of 0.");
+  Preconditions.checkArgument(outputBuffer.position() == 0, "Invalid 
position of 0.");
+  // There is compressed input, decompress it now.
+  int decompressedSize = uncompressedLength(inputBuffer, len);
+  if (decompressedSize > outputBuffer.capacity()) {
+ByteBuffer oldBuffer = outputBuffer;
+outputBuffer = ByteBuffer.allocateDirect(decompressedSize);
+CleanUtil.cleanDirectBuffer(oldBuffer);
+  }
+
+  // Reset the previous outputBuffer (i.e. set position to 0)
+  outputBuffer.clear();
+  int size = uncompress(inputBuffer, outputBuffer);
+  outputBuffer.limit(size);
+  // We've decompressed the entire input, reset the input now
+  inputBuffer.clear();
+  inputBuffer.limit(0);
+  finished = true;
+}
+
+// Return compressed output up to 'len'
+int numBytes = Math.min(len, outputBuffer.remaining());
+outputBuffer.get(buffer, off, numBytes);
+return numBytes;
+  }
+
+  /**
+   * Sets input data for decompression.
+   * This should be called if and only if {@link #needsInput()} returns
+   * true indicating that more input data is required.
+   * (Both native and non-native versions of various Decompressors require
+   * that the data passed in via b[] remain unmodified until
+   * the caller is explicitly notified--via {@link #needsInput()}--that the
+   * buffer may be safely modified.  With this requirement, an extra
+   * buffer-copy can be avoided.)
+   *
+   * @param buffer Input data
+   * @param offStart offset
+   * @param lenLength
+   */
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {
+SnappyUtil.validateBuffer(buffer, off, len);
+
+if (inputBuffer.capacity() - inputBuffer.position() < len) {
+  final ByteBuffer newBuffer = 
ByteBuffer.allocateDirect(inputBuffer.position() + len);
+  inputBuffer.rewind();
+  newBuffer.put(inputBuffer);
+  final ByteBuffer oldBuffer = inputBuffer;
+  inputBuffer = newBuffer;
+  CleanUtil.cleanDirectBuffer(oldBuffer);
+} else {
+  inputBuffer.limit(inputBuffer.position() + len);
+}
+

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/NonBlockedDecompressor.java:
##
@@ -0,0 +1,174 @@
+/*
+ * 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.io.compress.Decompressor;
+import org.apache.parquet.Preconditions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+abstract public class NonBlockedDecompressor implements Decompressor {
+
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer Buffer for the compressed data
+   * @param offStart offset of the data
+   * @param lenSize of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException if reading or decompression fails
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws 
IOException {
+SnappyUtil.validateBuffer(buffer, off, len);
+if (inputBuffer.position() == 0 && !outputBuffer.hasRemaining()) {
+  return 0;
+}
+
+if (!outputBuffer.hasRemaining()) {
+  inputBuffer.rewind();
+  Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid 
position of 0.");
+  Preconditions.checkArgument(outputBuffer.position() == 0, "Invalid 
position of 0.");
+  // There is compressed input, decompress it now.
+  int decompressedSize = uncompressedLength(inputBuffer, len);
+  if (decompressedSize > outputBuffer.capacity()) {
+ByteBuffer oldBuffer = outputBuffer;
+outputBuffer = ByteBuffer.allocateDirect(decompressedSize);
+CleanUtil.cleanDirectBuffer(oldBuffer);
+  }
+
+  // Reset the previous outputBuffer (i.e. set position to 0)
+  outputBuffer.clear();
+  int size = uncompress(inputBuffer, outputBuffer);
+  outputBuffer.limit(size);
+  // We've decompressed the entire input, reset the input now
+  inputBuffer.clear();
+  inputBuffer.limit(0);
+  finished = true;
+}
+
+// Return compressed output up to 'len'
+int numBytes = Math.min(len, outputBuffer.remaining());
+outputBuffer.get(buffer, off, numBytes);
+return numBytes;
+  }
+
+  /**
+   * Sets input data for decompression.
+   * This should be called if and only if {@link #needsInput()} returns
+   * true indicating that more input data is required.
+   * (Both native and non-native versions of various Decompressors require
+   * that the data passed in via b[] remain unmodified until
+   * the caller is explicitly notified--via {@link #needsInput()}--that the
+   * buffer may be safely modified.  With this requirement, an extra
+   * buffer-copy can be avoided.)
+   *
+   * @param buffer Input data
+   * @param offStart offset
+   * @param lenLength
+   */
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {
+SnappyUtil.validateBuffer(buffer, off, len);

Review Comment:
   Should we refactor this SnappyUtil also? 





> 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 

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawDecompressor.java:
##
@@ -0,0 +1,46 @@
+/*
+ * 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 io.airlift.compress.lz4.Lz4Decompressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawDecompressor extends NonBlockedDecompressor {
+
+  private Lz4Decompressor decompressor = new Lz4Decompressor();
+
+  @Override
+  protected int uncompress(ByteBuffer compressed, ByteBuffer uncompressed) 
throws IOException {
+decompressor.decompress(compressed, uncompressed);
+int uncompressedSize = uncompressed.position();
+uncompressed.limit(uncompressedSize);
+uncompressed.rewind();
+return uncompressedSize;
+  }
+
+  @Override
+  protected int uncompressedLength(ByteBuffer compressed, int 
maxUncompressedLength) throws IOException {

Review Comment:
   Are you intentionally for uncompressedMaxLength?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawDecompressor.java:
##
@@ -0,0 +1,46 @@
+/*
+ * 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 io.airlift.compress.lz4.Lz4Decompressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawDecompressor extends NonBlockedDecompressor {
+
+  private Lz4Decompressor decompressor = new Lz4Decompressor();
+
+  @Override
+  protected int uncompress(ByteBuffer compressed, ByteBuffer uncompressed) 
throws IOException {

Review Comment:
   For the parity of the two files(LZ4RawCompressor.java and 
LZ4RawDecompressor.java), can you follow the same order of the methods 
xxxLength()xxcompress()? This is minor issue though. 





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();
+
+  @Override
+  protected int maxCompressedLength(int byteSize) {
+return 
io.airlift.compress.lz4.Lz4RawCompressor.maxCompressedLength(byteSize);

Review Comment:
   use import



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();
+
+  @Override
+  protected int maxCompressedLength(int byteSize) {
+return 
io.airlift.compress.lz4.Lz4RawCompressor.maxCompressedLength(byteSize);

Review Comment:
   use import?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,99 @@
+/*
+ * 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.*;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0.
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+  // Hadoop config for how big to make intermediate buffers.
+  private final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  @Override
+  public void setConf(Configuration conf) {
+this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+return conf;
+  }
+
+  @Override
+  public Compressor createCompressor() {
+return new Lz4RawCompressor();
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+return new Lz4RawDecompressor();
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream)
+throws IOException {
+return createInputStream(stream, createDecompressor());
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream,
+  Decompressor decompressor) 
throws IOException {
+return new NonBlockedDecompressorStream(stream, decompressor,
+  conf.getInt(BUFFER_SIZE_CONFIG, 4*1024));

Review Comment:
   Can you make it a DEFAULT_ BUFFER_SIZE_CONFIG?





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,99 @@
+/*
+ * 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.*;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Lz4 raw compression codec for Parquet. This codec type has been introduced
+ * into the parquet format since version 2.9.0.
+ */
+public class Lz4RawCodec implements Configurable, CompressionCodec {
+
+  private Configuration conf;
+  // Hadoop config for how big to make intermediate buffers.
+  private final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";

Review Comment:
   Can we make it clear the unit of the size





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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1259621617

   Thank Gang for contributing! Is there any benchmarking numbers? Any 
comparison with ZSTD? These are non-blocking question for review and merging. 




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1259431976

   > @wgtmac Did you try to read an actual file produced by Parquet C++?
   > 
   > Note you can find such files in https://github.com/apache/parquet-testing/
   
   Yes, I have tried that. I will add some parquet files for compatibility test 
as well.




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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, 

[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

pitrou commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1259279363

   cc @lidavidm




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac commented on PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#issuecomment-1259233215

   @pitrou @shangxinli Can you please take a look? Thanks in advance!




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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

2022-09-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2196:
-

wgtmac opened a new pull request, #1000:
URL: https://github.com/apache/parquet-mr/pull/1000

   This PR implements the LZ4_RAW codec which was introduced by parquet format 
v2.9.0. Since there are a lot of common logic between the LZ4_RAW and SNAPPY 
codecs, this patch moves them into NonBlockedCompressor and 
NonBlockedDecompressor and make the specific codec extend them.
   
   Added TestLz4RawCodec test to make sure the new codec itself is correct.




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