[jira] [Commented] (PARQUET-1948) TransCompressionCommand Inoperable

2021-02-18 Thread Xinli Shang (Jira)


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

Xinli Shang commented on PARQUET-1948:
--

[~vanhooser], glad to see you have the interests of this tool. We have been 
using it by translating GZIP to ZSTD for existing parquet files. Let me know if 
you hit any issues. 

> TransCompressionCommand Inoperable
> --
>
> Key: PARQUET-1948
> URL: https://issues.apache.org/jira/browse/PARQUET-1948
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.11.1
> Environment: I am using parquet-tools 1.11.1 on a Mac machine running 
> Catalina, and my parquet-tools jar was downloaded from Maven Central. 
>Reporter: Shelby Vanhooser
>Priority: Blocker
>  Labels: parquet-tools
>
> {{TransCompressionCommand}} in parquet-tools is intended to allow translation 
> of compression types in parquet files.  We are intending to use this 
> functionality to debug a corrupted file, but this command fails to run at the 
> moment entirely. 
> Running the following command (on the uncorrupted file):
> {code:java}
> java -jar ./parquet-tools-1.11.1.jar trans-compression 
> ~/Downloads/part-00048-69f65188-94b5-4772-8906-5c78989240b5_00048.c000.snappy.parquet{code}
> This results in 
>  
> {code:java}
> Unknown command: trans-compression{code}
>  
> I believe this is due to the Registry class [silently catching any errors to 
> initialize|https://github.com/apache/parquet-mr/blob/master/parquet-tools/src/main/java/org/apache/parquet/tools/command/Registry.java#L65]
>  which subsequently is [misinterpreted as an unknown 
> command|https://github.com/apache/parquet-mr/blob/master/parquet-tools/src/main/java/org/apache/parquet/tools/Main.java#L200].
> We need to: 
>  # Write a test for the TransCompressionCommand to figure out why it's 
> showing up as unknown command
>  # Probably expand these tests to cover all the other commands
>  
> This will then unblock our debugging work on the suspect file. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1982) Allow random access to row groups in ParquetFileReader

2021-02-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1982:
-

gszadovszky commented on a change in pull request #871:
URL: https://github.com/apache/parquet-mr/pull/871#discussion_r578525890



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -888,20 +888,44 @@ public void appendTo(ParquetFileWriter writer) throws 
IOException {
 writer.appendRowGroups(f, blocks, true);
   }
 
+  /**
+   * Reads all the columns requested from the row group at the specified block.
+   * @throws IOException if an error occurs while reading
+   * @return the PageReadStore which can provide PageReaders for each column.
+   */
+  public PageReadStore readRowGroup(BlockMetaData block) throws IOException {

Review comment:
   You have created the specific method to read a row group based on the 
specified metadata (instead simply reading the next row group). This seems fine 
to me. Meanwhile, there is another method for reading the next row group: 
`readNextFilteredRowGroup`. This method was created to support the column 
indexes feature and meanwhile to be backward compatible with the old readers.
   To take advantage of the column indexes feature (and skip the unnecessary 
pages to get read) and also for symmetry I would suggest creating another 
specific method: `readFilteredRowGroup(BlockMetaData)`.

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.parquet.column.Encoding.BIT_PACKED;
+import static org.apache.parquet.column.Encoding.PLAIN;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private static final MessageType SCHEMA = 
MessageTypeParser.parseMessageType("" +
+"message m {" +
+"  required group a {" +
+"required binary b;" +
+"  }" +
+"  required group c {" +
+"required int64 d;" +
+"  }" +
+"}");
+  private static final String[] PATH1 = {"a", "b"};
+  private static final ColumnDescriptor C1 = 
SCHEMA.getColumnDescription(PATH1);
+  private static final String[] PATH2 = {"c", "d"};
+  private static final ColumnDescriptor C2 = 
SCHEMA.getColumnDescription(PATH2);
+
+  private static final byte[] BYTES1 = { 0, 1, 2, 3 };
+  private static final byte[] BYTES2 = { 1, 2, 3, 4 };
+  private static final byte[] BYTES3 = { 2, 3, 4, 5 };
+  private static final byte[] BYTES4 = { 3, 4, 5, 6 };
+  private static final CompressionCodecName CODEC = 
CompressionCodecName.UNCOMPRESSED;
+
+  private static final org.apache.parquet.column.statistics.Statistics 
EMPTY_STATS = org.apache.parquet.column.statistics.Statistics
+

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #871: PARQUET-1982: Random access to row groups in ParquetFileReader

2021-02-18 Thread GitBox


gszadovszky commented on a change in pull request #871:
URL: https://github.com/apache/parquet-mr/pull/871#discussion_r578525890



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -888,20 +888,44 @@ public void appendTo(ParquetFileWriter writer) throws 
IOException {
 writer.appendRowGroups(f, blocks, true);
   }
 
+  /**
+   * Reads all the columns requested from the row group at the specified block.
+   * @throws IOException if an error occurs while reading
+   * @return the PageReadStore which can provide PageReaders for each column.
+   */
+  public PageReadStore readRowGroup(BlockMetaData block) throws IOException {

Review comment:
   You have created the specific method to read a row group based on the 
specified metadata (instead simply reading the next row group). This seems fine 
to me. Meanwhile, there is another method for reading the next row group: 
`readNextFilteredRowGroup`. This method was created to support the column 
indexes feature and meanwhile to be backward compatible with the old readers.
   To take advantage of the column indexes feature (and skip the unnecessary 
pages to get read) and also for symmetry I would suggest creating another 
specific method: `readFilteredRowGroup(BlockMetaData)`.

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.parquet.column.Encoding.BIT_PACKED;
+import static org.apache.parquet.column.Encoding.PLAIN;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private static final MessageType SCHEMA = 
MessageTypeParser.parseMessageType("" +
+"message m {" +
+"  required group a {" +
+"required binary b;" +
+"  }" +
+"  required group c {" +
+"required int64 d;" +
+"  }" +
+"}");
+  private static final String[] PATH1 = {"a", "b"};
+  private static final ColumnDescriptor C1 = 
SCHEMA.getColumnDescription(PATH1);
+  private static final String[] PATH2 = {"c", "d"};
+  private static final ColumnDescriptor C2 = 
SCHEMA.getColumnDescription(PATH2);
+
+  private static final byte[] BYTES1 = { 0, 1, 2, 3 };
+  private static final byte[] BYTES2 = { 1, 2, 3, 4 };
+  private static final byte[] BYTES3 = { 2, 3, 4, 5 };
+  private static final byte[] BYTES4 = { 3, 4, 5, 6 };
+  private static final CompressionCodecName CODEC = 
CompressionCodecName.UNCOMPRESSED;
+
+  private static final org.apache.parquet.column.statistics.Statistics 
EMPTY_STATS = org.apache.parquet.column.statistics.Statistics
+
.getBuilderForReading(Types.required(PrimitiveType.PrimitiveTypeName.BINARY).named("test_binary")).build();
+
+  private void writeDataToFile(Path path) throws IOException {
+Configuration configuration = new Configuration();
+
+ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path);
+w.start();
+w.startBlock(3);
+ 

[jira] [Commented] (PARQUET-1984) Some tests fail on windows

2021-02-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1984:
-

gszadovszky commented on a change in pull request #870:
URL: https://github.com/apache/parquet-mr/pull/870#discussion_r578513538



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
##
@@ -479,6 +479,7 @@ public void test() throws IOException {
   while ((pageReadStore = reader.readNextRowGroup()) != null) {
 validator.validate(metadata.getFileMetaData().getSchema(), 
pageReadStore);
   }
+  reader.close();

Review comment:
   I would suggest using try-with-resources to ensure reader is always 
closed

##
File path: 
parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestParquetToThriftReadWriteAndProjection.java
##
@@ -377,9 +377,9 @@ private void 
shouldDoProjectionWithThriftColumnFilter(String filterDesc, TBase t
 readValue = reader.getCurrentValue();
 LOG.info("{}", readValue);
   }
+  reader.close();

Review comment:
   Same as above

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestInputFormat.java
##
@@ -398,7 +399,10 @@ public void testGetFootersReturnsInPredictableOrder() 
throws IOException {
   if (i > 0) {
 url.append(',');
   }
-  url.append("file:").append(file.getAbsolutePath());
+  if (!SystemUtils.IS_OS_WINDOWS) {
+url.append("file:");
+  }
+  url.append(file.getAbsolutePath());

Review comment:
   I don't have any experience running Hadoop on Windows but I am not sure 
this is the good approach to pass the tests. I think it would be better to 
expect the same configuration on the different systems and handle OS relevant 
paths in the production code if required.





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.

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


> Some tests fail on windows
> --
>
> Key: PARQUET-1984
> URL: https://issues.apache.org/jira/browse/PARQUET-1984
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr, parquet-thrift
>Affects Versions: 1.12.0
> Environment: Windows 10
>Reporter: Felix Schmalzel
>Priority: Minor
>
> Reasons:
>  * Expecting \n and getting \r\n
>  * Unclosed streams preventing a temporary file from being deleted
>  * File layout differences \ and /
>  * No native library for brotli, because the brotli-codec dependency only 
> shadows macos and linux native libraries.
>  
> I've already developed a patch that would fix all the problems excluding the 
> brotli one. For that one we would have to wait until this 
> [https://github.com/rdblue/brotli-codec/pull/2] request is merged. I will 
> link the merge request for the other problems in the next few days.
> Is there a related ticket that i have overlooked?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #870: PARQUET-1984: Allow tests to run on windows

2021-02-18 Thread GitBox


gszadovszky commented on a change in pull request #870:
URL: https://github.com/apache/parquet-mr/pull/870#discussion_r578513538



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
##
@@ -479,6 +479,7 @@ public void test() throws IOException {
   while ((pageReadStore = reader.readNextRowGroup()) != null) {
 validator.validate(metadata.getFileMetaData().getSchema(), 
pageReadStore);
   }
+  reader.close();

Review comment:
   I would suggest using try-with-resources to ensure reader is always 
closed

##
File path: 
parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestParquetToThriftReadWriteAndProjection.java
##
@@ -377,9 +377,9 @@ private void 
shouldDoProjectionWithThriftColumnFilter(String filterDesc, TBase t
 readValue = reader.getCurrentValue();
 LOG.info("{}", readValue);
   }
+  reader.close();

Review comment:
   Same as above

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestInputFormat.java
##
@@ -398,7 +399,10 @@ public void testGetFootersReturnsInPredictableOrder() 
throws IOException {
   if (i > 0) {
 url.append(',');
   }
-  url.append("file:").append(file.getAbsolutePath());
+  if (!SystemUtils.IS_OS_WINDOWS) {
+url.append("file:");
+  }
+  url.append(file.getAbsolutePath());

Review comment:
   I don't have any experience running Hadoop on Windows but I am not sure 
this is the good approach to pass the tests. I think it would be better to 
expect the same configuration on the different systems and handle OS relevant 
paths in the production code if required.





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.

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




[jira] [Commented] (PARQUET-1975) Test failure on ARM64 CPU architecture

2021-02-18 Thread Martin Tzvetanov Grigorov (Jira)


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

Martin Tzvetanov Grigorov commented on PARQUET-1975:


[https://github.com/apache/parquet-mr/pull/872] could be used until a new 
version of brotli-codec is released.

So far there is no feedback on [https://github.com/rdblue/brotli-codec/pull/1] 
by [~rdblue]

> Test failure on ARM64 CPU architecture
> --
>
> Key: PARQUET-1975
> URL: https://issues.apache.org/jira/browse/PARQUET-1975
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Martin Tzvetanov Grigorov
>Priority: Minor
>
> Trying to build Apache Parquet MR on ARM64 fails with:
>  
> {code:java}
> $ mvn clean verify
> ...
> Tests in error: 
>   
> testReadWriteWithCountDeprecated(org.apache.parquet.hadoop.DeprecatedInputFormatTest):
>  org.apache.hadoop.io.compress.CompressionCodec: Provider 
> org.apache.hadoop.io.compress.BrotliCodec could not be instantiated
> {code}
>  
> The reason is that com.github.rdblue:brotli-codec has no binary for aarch64



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1975) Test failure on ARM64 CPU architecture

2021-02-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1975:
-

martin-g opened a new pull request #872:
URL: https://github.com/apache/parquet-mr/pull/872


   BrotliCodec fails to load on non-x86_64 CPU architectures because 
MeteoGroup/jbrotli does not provide native binaries
   With this change brotli-codec is added as a dependency to parquet-hadoop 
only for amd64, i.e. x86_64



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.

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


> Test failure on ARM64 CPU architecture
> --
>
> Key: PARQUET-1975
> URL: https://issues.apache.org/jira/browse/PARQUET-1975
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Martin Tzvetanov Grigorov
>Priority: Minor
>
> Trying to build Apache Parquet MR on ARM64 fails with:
>  
> {code:java}
> $ mvn clean verify
> ...
> Tests in error: 
>   
> testReadWriteWithCountDeprecated(org.apache.parquet.hadoop.DeprecatedInputFormatTest):
>  org.apache.hadoop.io.compress.CompressionCodec: Provider 
> org.apache.hadoop.io.compress.BrotliCodec could not be instantiated
> {code}
>  
> The reason is that com.github.rdblue:brotli-codec has no binary for aarch64



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] martin-g opened a new pull request #872: PARQUET-1975 Disable BrotliCodec for non-x86_64

2021-02-18 Thread GitBox


martin-g opened a new pull request #872:
URL: https://github.com/apache/parquet-mr/pull/872


   BrotliCodec fails to load on non-x86_64 CPU architectures because 
MeteoGroup/jbrotli does not provide native binaries
   With this change brotli-codec is added as a dependency to parquet-hadoop 
only for amd64, i.e. x86_64



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.

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




[jira] [Commented] (PARQUET-1982) Allow random access to row groups in ParquetFileReader

2021-02-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1982:
-

fschmalzel opened a new pull request #871:
URL: https://github.com/apache/parquet-mr/pull/871


   Adds a method readRowGroup(BlockMetaData) to allow random access to
   PageReadStores via BlockMetaData, which can be obtained using the
   getRowGroups() method.
   
   This is similar to the existing method
   getDictionaryReader(BlockMetaData)
   that already exists.
   
   With random access the reader can be reused if for example someone
   needs to go back a row group. This would improve performance
   because we don't need to open the file again and read the metadata.
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [x ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1982) issue.
   
   ### Tests
   
   - [x ] My PR adds the following unit tests:
- TestParquetReaderRandomAccess
   
   ### Commits
   
   - [x ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [x ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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.

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


> Allow random access to row groups in ParquetFileReader
> --
>
> Key: PARQUET-1982
> URL: https://issues.apache.org/jira/browse/PARQUET-1982
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-mr
>Reporter: Felix Schmalzel
>Priority: Minor
>  Labels: parquetReader, random-access
>
> The used SeekableInputStream and all other components of the 
> ParquetFileReader already support random access and row groups should be 
> independent of each other.
> This would allow reusing the opened InputStream when you want to go back a 
> row group. It also makes accessing specific row groups a lot easier.
> I've already developed a patch that would enable this functionality. I will 
> link the merge request in the next few days.
> Is there a related ticket that i have overlooked?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] fschmalzel opened a new pull request #871: PARQUET-1982: Random access to row groups in ParquetFileReader

2021-02-18 Thread GitBox


fschmalzel opened a new pull request #871:
URL: https://github.com/apache/parquet-mr/pull/871


   Adds a method readRowGroup(BlockMetaData) to allow random access to
   PageReadStores via BlockMetaData, which can be obtained using the
   getRowGroups() method.
   
   This is similar to the existing method
   getDictionaryReader(BlockMetaData)
   that already exists.
   
   With random access the reader can be reused if for example someone
   needs to go back a row group. This would improve performance
   because we don't need to open the file again and read the metadata.
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [x ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1982) issue.
   
   ### Tests
   
   - [x ] My PR adds the following unit tests:
- TestParquetReaderRandomAccess
   
   ### Commits
   
   - [x ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [x ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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.

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




[jira] [Commented] (PARQUET-1984) Some tests fail on windows

2021-02-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1984:
-

fschmalzel opened a new pull request #870:
URL: https://github.com/apache/parquet-mr/pull/870


   Check for \r\n lineendings instead of \n
   Change file layout (backslash and slash) to check
   Close files / streams before deleting file
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1984) issue.
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason: It fixes tests for the windows platform.
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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.

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


> Some tests fail on windows
> --
>
> Key: PARQUET-1984
> URL: https://issues.apache.org/jira/browse/PARQUET-1984
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr, parquet-thrift
>Affects Versions: 1.12.0
> Environment: Windows 10
>Reporter: Felix Schmalzel
>Priority: Minor
>
> Reasons:
>  * Expecting \n and getting \r\n
>  * Unclosed streams preventing a temporary file from being deleted
>  * File layout differences \ and /
>  * No native library for brotli, because the brotli-codec dependency only 
> shadows macos and linux native libraries.
>  
> I've already developed a patch that would fix all the problems excluding the 
> brotli one. For that one we would have to wait until this 
> [https://github.com/rdblue/brotli-codec/pull/2] request is merged. I will 
> link the merge request for the other problems in the next few days.
> Is there a related ticket that i have overlooked?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] fschmalzel opened a new pull request #870: PARQUET-1984: Allow tests to run on windows

2021-02-18 Thread GitBox


fschmalzel opened a new pull request #870:
URL: https://github.com/apache/parquet-mr/pull/870


   Check for \r\n lineendings instead of \n
   Change file layout (backslash and slash) to check
   Close files / streams before deleting file
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1984) issue.
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason: It fixes tests for the windows platform.
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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.

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




[jira] [Commented] (PARQUET-1985) Improve integration tests between implementations

2021-02-18 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou commented on PARQUET-1985:
-

Indeed, JSON sounds much better than CSV.

We probably want to test logical types as well, IMHO. Of course, 
implementations which don't support them will have to skip those tests. Meaning 
optional features such as logical types must probably use separate reference 
files.


> Improve integration tests between implementations
> -
>
> Key: PARQUET-1985
> URL: https://issues.apache.org/jira/browse/PARQUET-1985
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-testing
> Environment: {noformat}
> *no* further _formatting_ is done here
> {noformat}
>Reporter: Gabor Szadovszky
>Priority: Major
>
> We have a lack of proper integration tests between components. Fortunately, 
> we already have a git repository to upload test data: 
> https://github.com/apache/parquet-testing.
> The idea is the following.
> Create a directory structure for the different versions of the 
> implementations containing parquet files with defined data. The structure 
> definition shall be self-descriptive so we can write integration tests that 
> reads the whole structure automatically and also works with files to be added 
> later.
> The following directory structure is an example for the previous requirements:
> {noformat}
> test-data/
> ├── impala
> │   ├── 3.2.0
> │   │   └── basic-data.parquet
> │   ├── 3.3.0
> │   │   └── basic-data.parquet
> │   └── 3.4.0
> │   ├── basic-data.lz4.parquet
> │   ├── basic-data.snappy.parquet
> │   ├── some-specific-issue-2.parquet
> │   ├── some-specific-issue-3.csv
> │   ├── some-specific-issue-3_mode1.parquet
> │   ├── some-specific-issue-3_mode2.parquet
> │   └── some-specific-issue-3.schema
> ├── parquet-cpp
> │   ├── 1.5.0
> │   │   ├── basic-data.lz4.parquet
> │   │   └── basic-data.parquet
> │   └── 1.6.0
> │   ├── basic-data.lz4.parquet
> │   └── some-specific-issue-2.parquet
> ├── parquet-mr
> │   ├── 1.10.2
> │   │   └── basic-data.parquet
> │   ├── 1.11.1
> │   │   ├── basic-data.parquet
> │   │   └── some-specific-issue-1.parquet
> │   ├── 1.12.0
> │   │   ├── basic-data.br.parquet
> │   │   ├── basic-data.lz4.parquet
> │   │   ├── basic-data.snappy.parquet
> │   │   ├── basic-data.zstd.parquet
> │   │   ├── some-specific-issue-1.parquet
> │   │   └── some-specific-issue-2.parquet
> │   ├── some-specific-issue-1.csv
> │   └── some-specific-issue-1.schema
> ├── basic-data.csv
> ├── basic-data.schema
> ├── some-specific-issue-2.csv
> └── some-specific-issue-2.schema
> {noformat}
> Parquet files are created at leaf level. The expected data is saved in a csv 
> format (to be specified: separators, how to save binary etc.), the expected 
> schema (to specify the data types independently from the parquet files) are 
> saved in .schema files. The csv and schema files can be saved on the same 
> level of the parquet files or upper levels if they are common to several 
> parquet files.
> Any comments about the idea are welcomed. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1985) Improve integration tests between implementations

2021-02-18 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1985:
---

[~emkornfield], I agree CSV is not the best approach. I did not think about 
nested types. I think JSON is more wide-spread than Protobuf or Avro so it has 
a higher chance to get an easy to use library on for language. In addition JSON 
is human readable making debugging easier (for non-binary types). Meanwhile, 
JSON would be much larger than Protobuf/Avro files. 

We might use any formats to store "gold data" we still need to properly specify 
the way. Do we want to test logical types as well? From e.g. Arrow/Impala point 
of view it make sense as they have the related types (e.g. timestamp, decimal). 
To validate these types we need to have data in they rich form (e.g. as a 
timestamp/decimal and not binary). Meanwhile, parquet-mr does not have support 
for these types so when we convert the binary values to these types we are not 
testing parquet-mr but the test itself. But maybe it is a parquet-mr related 
issue and we shall provide the widest set of data available.

> Improve integration tests between implementations
> -
>
> Key: PARQUET-1985
> URL: https://issues.apache.org/jira/browse/PARQUET-1985
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-testing
> Environment: {noformat}
> *no* further _formatting_ is done here
> {noformat}
>Reporter: Gabor Szadovszky
>Priority: Major
>
> We have a lack of proper integration tests between components. Fortunately, 
> we already have a git repository to upload test data: 
> https://github.com/apache/parquet-testing.
> The idea is the following.
> Create a directory structure for the different versions of the 
> implementations containing parquet files with defined data. The structure 
> definition shall be self-descriptive so we can write integration tests that 
> reads the whole structure automatically and also works with files to be added 
> later.
> The following directory structure is an example for the previous requirements:
> {noformat}
> test-data/
> ├── impala
> │   ├── 3.2.0
> │   │   └── basic-data.parquet
> │   ├── 3.3.0
> │   │   └── basic-data.parquet
> │   └── 3.4.0
> │   ├── basic-data.lz4.parquet
> │   ├── basic-data.snappy.parquet
> │   ├── some-specific-issue-2.parquet
> │   ├── some-specific-issue-3.csv
> │   ├── some-specific-issue-3_mode1.parquet
> │   ├── some-specific-issue-3_mode2.parquet
> │   └── some-specific-issue-3.schema
> ├── parquet-cpp
> │   ├── 1.5.0
> │   │   ├── basic-data.lz4.parquet
> │   │   └── basic-data.parquet
> │   └── 1.6.0
> │   ├── basic-data.lz4.parquet
> │   └── some-specific-issue-2.parquet
> ├── parquet-mr
> │   ├── 1.10.2
> │   │   └── basic-data.parquet
> │   ├── 1.11.1
> │   │   ├── basic-data.parquet
> │   │   └── some-specific-issue-1.parquet
> │   ├── 1.12.0
> │   │   ├── basic-data.br.parquet
> │   │   ├── basic-data.lz4.parquet
> │   │   ├── basic-data.snappy.parquet
> │   │   ├── basic-data.zstd.parquet
> │   │   ├── some-specific-issue-1.parquet
> │   │   └── some-specific-issue-2.parquet
> │   ├── some-specific-issue-1.csv
> │   └── some-specific-issue-1.schema
> ├── basic-data.csv
> ├── basic-data.schema
> ├── some-specific-issue-2.csv
> └── some-specific-issue-2.schema
> {noformat}
> Parquet files are created at leaf level. The expected data is saved in a csv 
> format (to be specified: separators, how to save binary etc.), the expected 
> schema (to specify the data types independently from the parquet files) are 
> saved in .schema files. The csv and schema files can be saved on the same 
> level of the parquet files or upper levels if they are common to several 
> parquet files.
> Any comments about the idea are welcomed. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)