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

Reply via email to