[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


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? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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);
+}
+inputBuffer.put(buffer, off, len);
+  }
+
+  @Override
+  public void end() {
+CleanUtil.cleanDirectBuffer(inputBuffer);
+CleanUtil.cleanDirectBuffer(outputBuffer);
+  }
+
+  @Override
+  public synchronized boolean finished() {
+return 

[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-27 Thread GitBox


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org