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