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

2022-09-29 Thread GitBox


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?



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



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

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

2022-09-29 Thread GitBox


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?



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



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


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

2022-09-29 Thread GitBox


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.



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



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


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

2022-09-29 Thread GitBox


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?



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



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


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

2022-09-29 Thread GitBox


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.



-- 
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] emkornfield commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-29 Thread GitBox


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.



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



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


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

2022-09-29 Thread GitBox


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?



-- 
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] emkornfield commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-29 Thread GitBox


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?



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



[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-1222) Specify a well-defined sorting order for float and double types

2022-09-29 Thread Micah Kornfield (Jira)


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

Micah Kornfield commented on PARQUET-1222:
--

I'd propose the following "fix":
- Add a new optional bool value to the statistics  struct "contains_nan".  When 
unset, I think we specify the semantics for comparisons relative to -0.0/0.0 
and NaN, etc are not well defined and implementations have taken different 
routes.
- When set, if true, it means the column contains at least one NaN, when set to 
false it means no NaNs are present.  Further when set, it implies the following 
ordering:
NaNs are never included in Min/Max statistics in the struct.  -0.0, +0.0, are 
considered two distinct values and are ordered according to sign.

Thoughts?  Should I bring this up on the mailing list?

> Specify a well-defined sorting order for float and double types
> ---
>
> Key: PARQUET-1222
> URL: https://issues.apache.org/jira/browse/PARQUET-1222
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Zoltan Ivanfi
>Priority: Critical
>
> Currently parquet-format specifies the sort order for floating point numbers 
> as follows:
> {code:java}
>*   FLOAT - signed comparison of the represented value
>*   DOUBLE - signed comparison of the represented value
> {code}
> The problem is that the comparison of floating point numbers is only a 
> partial ordering with strange behaviour in specific corner cases. For 
> example, according to IEEE 754, -0 is neither less nor more than \+0 and 
> comparing NaN to anything always returns false. This ordering is not suitable 
> for statistics. Additionally, the Java implementation already uses a 
> different (total) ordering that handles these cases correctly but differently 
> than the C\+\+ implementations, which leads to interoperability problems.
> TypeDefinedOrder for doubles and floats should be deprecated and a new 
> TotalFloatingPointOrder should be introduced. The default for writing doubles 
> and floats would be the new TotalFloatingPointOrder. This ordering should be 
> effective and easy to implement in all programming languages.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-758) [Format] HALF precision FLOAT Logical type

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


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

ASF GitHub Bot commented on PARQUET-758:


emkornfield commented on PR #184:
URL: https://github.com/apache/parquet-format/pull/184#issuecomment-1263116115

   Sorry for the delay, it sounds like PARQUET-1222 is blocker, let me make a 
proposal there and see if we can at least come to consensus on approach and 
maybe this feature can be the first test-case for it.




> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Minor
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-format] emkornfield commented on pull request #184: PARQUET-758: Add Float16/Half-float logical type

2022-09-29 Thread GitBox


emkornfield commented on PR #184:
URL: https://github.com/apache/parquet-format/pull/184#issuecomment-1263116115

   Sorry for the delay, it sounds like PARQUET-1222 is blocker, let me make a 
proposal there and see if we can at least come to consensus on approach and 
maybe this feature can be the first test-case for it.


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



[jira] [Commented] (PARQUET-758) [Format] HALF precision FLOAT Logical type

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


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

ASF GitHub Bot commented on PARQUET-758:


anjakefala commented on PR #184:
URL: https://github.com/apache/parquet-format/pull/184#issuecomment-1262879211

   @pitrou @emkornfield @gszadovszky 
   
   Is there anything I can do to move this addition forward? Can I help with 
any code?
   
   My understanding from reading the comments is that @gszadovszky brought up 
an ordering concern (valid, but not a blocker?), and that a decision needs to 
be made on whether float16 would be implemented as a logical or physical type? 




> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Minor
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-format] anjakefala commented on pull request #184: PARQUET-758: Add Float16/Half-float logical type

2022-09-29 Thread GitBox


anjakefala commented on PR #184:
URL: https://github.com/apache/parquet-format/pull/184#issuecomment-1262879211

   @pitrou @emkornfield @gszadovszky 
   
   Is there anything I can do to move this addition forward? Can I help with 
any code?
   
   My understanding from reading the comments is that @gszadovszky brought up 
an ordering concern (valid, but not a blocker?), and that a decision needs to 
be made on whether float16 would be implemented as a logical or physical type? 


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



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-29 Thread GitBox


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 


-- 
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] ggershinsky commented on pull request #962: Performance optimization to ByteBitPackingValuesReader

2022-09-29 Thread GitBox


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

   Optimizations like using byte arrays instead of byte buffers, and allocating 
the byte array once only, instead of per operation. Done in a concise manner, 
without unnecessary code changes. LGTM.


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



[jira] [Commented] (PARQUET-2193) Encrypting only one field in nested field prevents reading of other fields in nested field without keys

2022-09-29 Thread Gidon Gershinsky (Jira)


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

Gidon Gershinsky commented on PARQUET-2193:
---

Hmm, looks like this method runs over all columns, projected and not projected:
org.apache.parquet.hadoop.ParquetRecordReader.checkDeltaByteArrayProblem(ParquetRecordReader.java:191)
 

Please check if setting "parquet.split.files" to "false" solves this problem.

> Encrypting only one field in nested field prevents reading of other fields in 
> nested field without keys
> ---
>
> Key: PARQUET-2193
> URL: https://issues.apache.org/jira/browse/PARQUET-2193
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Vignesh Nageswaran
>Priority: Major
>
> Hi Team,
> While exploring parquet encryption, it is found that, if a field in nested 
> column is encrypted , and If I want to read this parquet directory from other 
> applications which does not have encryption keys to decrypt it, I cannot read 
> the remaining fields of the nested column without keys. 
> Example 
> `
> {code:java}
> case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
> case class SquareItem(int_column: Int, square_int_column : Double, 
> partitionCol: Int, nestedCol :nestedItem)
> `{code}
> In the case class `SquareItem` , `nestedCol` field is nested field and I want 
> to encrypt a field `ic` within it. 
>  
> I also want the footer to be non encrypted , so that I can use the encrypted 
> parquet file by legacy applications. 
>  
> Encryption is successful, however, when I query the parquet file using spark 
> 3.3.0 without having any configuration for parquet encryption set up , I 
> cannot non encrypted fields of `nestedCol` `sic`. I was expecting that only 
> `nestedCol` `ic` field will not be querable.
>  
>  
> Reproducer. 
> Spark 3.3.0 Using Spark-shell 
> Downloaded the file 
> [parquet-hadoop-1.12.0-tests.jar|https://repo1.maven.org/maven2/org/apache/parquet/parquet-hadoop/1.12.0/parquet-hadoop-1.12.0-tests.jar]
>  and added it to spark-jars folder
> Code to create encrypted data. #  
>  
> {code:java}
> sc.hadoopConfiguration.set("parquet.crypto.factory.class" 
> ,"org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory")
> sc.hadoopConfiguration.set("parquet.encryption.kms.client.class" 
> ,"org.apache.parquet.crypto.keytools.mocks.InMemoryKMS")
> sc.hadoopConfiguration.set("parquet.encryption.key.list","key1a: 
> BAECAwQFBgcICQoLDA0ODw==, key2a: BAECAAECAAECAAECAAECAA==, keyz: 
> BAECAAECAAECAAECAAECAA==")
> sc.hadoopConfiguration.set("parquet.encryption.key.material.store.internally","false")
> val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
> valpartitionCol = 1
> case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
> case class SquareItem(int_column: Int, square_int_column : Double, 
> partitionCol: Int, nestedCol :nestedItem)
> val dataRange = (1 to 100).toList
> val squares = sc.parallelize(dataRange.map(i => new SquareItem(i, 
> scala.math.pow(i,2), partitionCol,nestedItem(i,i
> squares.toDS().show()
> squares.toDS().write.partitionBy("partitionCol").mode("overwrite").option("parquet.encryption.column.keys",
>  
> "key1a:square_int_column,nestedCol.ic;").option("parquet.encryption.plaintext.footer",true).option("parquet.encryption.footer.key",
>  "keyz").parquet(encryptedParquetPath)
> {code}
> Code to read the data trying to access non encrypted nested field by opening 
> a new spark-shell
>  
> {code:java}
> val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
> spark.sqlContext.read.parquet(encryptedParquetPath).createOrReplaceTempView("test")
> spark.sql("select nestedCol.sic from test").show(){code}
> As you can see that nestedCol.sic is not encrypted , I was expecting the 
> results, but
> I get the below error
>  
> {code:java}
> Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: 
> [square_int_column]. Null File Decryptor
>   at 
> org.apache.parquet.hadoop.metadata.EncryptedColumnChunkMetaData.decryptIfNeeded(ColumnChunkMetaData.java:602)
>   at 
> org.apache.parquet.hadoop.metadata.ColumnChunkMetaData.getEncodings(ColumnChunkMetaData.java:348)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.checkDeltaByteArrayProblem(ParquetRecordReader.java:191)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:177)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>   at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.$anonfun$buildReaderWithPartitionValues$1(ParquetFileFormat.scala:375)
>   at 
> 

[jira] [Assigned] (PARQUET-2187) Add Parquet file containing a boolean column with RLE encoding to paquet

2022-09-29 Thread Antoine Pitrou (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Antoine Pitrou reassigned PARQUET-2187:
---

Assignee: Nishanth

> Add Parquet file containing a boolean column with RLE encoding to paquet
> 
>
> Key: PARQUET-2187
> URL: https://issues.apache.org/jira/browse/PARQUET-2187
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-testing
>Reporter: Nishanth
>Assignee: Nishanth
>Priority: Minor
>  Labels: pull-request-available
>
> Precursor to https://issues.apache.org/jira/browse/ARROW-17450 . 
> Add a test file in parquet-testing containing a boolean column with RLE 
> encoding. 
> The test files will be used by parquet implementation to validate the 
> encoding can be read 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (PARQUET-2187) Add Parquet file containing a boolean column with RLE encoding to paquet

2022-09-29 Thread Antoine Pitrou (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Antoine Pitrou resolved PARQUET-2187.
-
Resolution: Fixed

> Add Parquet file containing a boolean column with RLE encoding to paquet
> 
>
> Key: PARQUET-2187
> URL: https://issues.apache.org/jira/browse/PARQUET-2187
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-testing
>Reporter: Nishanth
>Priority: Minor
>  Labels: pull-request-available
>
> Precursor to https://issues.apache.org/jira/browse/ARROW-17450 . 
> Add a test file in parquet-testing containing a boolean column with RLE 
> encoding. 
> The test files will be used by parquet implementation to validate the 
> encoding can be read 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2194) parquet.encryption.plaintext.footer parameter being true, code expects parquet.encryption.footer.key

2022-09-29 Thread Gidon Gershinsky (Jira)


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

Gidon Gershinsky commented on PARQUET-2194:
---

Footer key is required also in the plaintext footer mode - it is used to sign 
the footer, 
https://github.com/apache/parquet-mr/tree/master/parquet-hadoop#class-propertiesdrivencryptofactory

> parquet.encryption.plaintext.footer parameter being true, code expects 
> parquet.encryption.footer.key
> 
>
> Key: PARQUET-2194
> URL: https://issues.apache.org/jira/browse/PARQUET-2194
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Vignesh Nageswaran
>Priority: Major
>
> Hi Team,
> I want my footer in parquet file to be non encrypted. so I set the 
> _parquet.encryption.plaintext.footer_ to be {_}true{_}, but when I tried to 
> run my code, parquet-mr is expecting __ value __ for the __ property 
> _parquet.encryption.footer.key  **_  
> Reproducer
> Spark 3.3.0 
> Download the 
> [file|[https://repo1.maven.org/maven2/org/apache/parquet/parquet-hadoop/1.12.0/parquet-hadoop-1.12.0-tests.jar]
>  ] and place it in spark - jar directory 
> using spark-shell
> {code:java}
> sc.hadoopConfiguration.set("parquet.crypto.factory.class" 
> ,"org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory") 
> sc.hadoopConfiguration.set("parquet.encryption.kms.client.class" 
> ,"org.apache.parquet.crypto.keytools.mocks.InMemoryKMS") 
> sc.hadoopConfiguration.set("parquet.encryption.key.list","key1a: 
> BAECAwQFBgcICQoLDA0ODw==, key2a: BAECAAECAAECAAECAAECAA==, keyz: 
> BAECAAECAAECAAECAAECAA==") 
> sc.hadoopConfiguration.set("parquet.encryption.key.material.store.internally","false")
>  
> val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted" 
> val partitionCol = 1 
> case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0) 
> case class SquareItem(int_column: Int, square_int_column : Double, 
> partitionCol: Int, nestedCol :nestedItem) 
> val dataRange = (1 to 100).toList 
> val squares = sc.parallelize(dataRange.map(i => new SquareItem(i, 
> scala.math.pow(i,2), partitionCol,nestedItem(i,i 
> squares.toDS().show() 
> squares.toDS().write.partitionBy("partitionCol").mode("overwrite").option("parquet.encryption.column.keys",
>  
> "key1a:square_int_column,nestedCol.ic;").option("parquet.encryption.plaintext.footer",true).parquet(encryptedParquetPath){code}
> I get the below error, my expectation is if I set properties for my footer to 
> be plain text, why do we need keys for footer.
>  
> {code:java}
>  
> Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: Undefined 
> footer key
>   at 
> org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory.getFileEncryptionProperties(PropertiesDrivenCryptoFactory.java:88)
>   at 
> org.apache.parquet.hadoop.ParquetOutputFormat.createEncryptionProperties(ParquetOutputFormat.java:554)
>   at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:478)
>   at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>   at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>   at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>   at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:155)
>   at 
> org.apache.spark.sql.execution.datasources.BaseDynamicPartitionDataWriter.renewCurrentWriter(FileFormatDataWriter.scala:298)
>   at 
> org.apache.spark.sql.execution.datasources.DynamicPartitionDataSingleWriter.write(FileFormatDataWriter.scala:365)
>   at 
> org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithMetrics(FileFormatDataWriter.scala:85)
>   at 
> org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithIterator(FileFormatDataWriter.scala:92)
>   at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:331)
>   at 
> org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1538)
>   at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:338)
>   ... 9 more
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)