[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
##
@@ -239,6 +248,82 @@ public void testWriteRead() throws Exception {
 PrintFooter.main(new String[] {path.toString()});
   }
 
+  @Test
+  public void testWriteReadWithRecordReader() throws Exception {

Review comment:
   I think this unit test does not reproduce the original scenario. Please 
check my comment in the 
[jira](https://issues.apache.org/jira/browse/PARQUET-2078?focusedCommentId=17406621=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17406621).




-- 
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-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], I am not sure how it would be possible. RowGroup.file_offset is set 
by using the dictionary page offset of the first column chunk (if there is any):
 * 
[rowGroup.setFile_offset(block.getStartingPos())|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L580]
 * 
[BlockMetaData.getStartingPos()|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/BlockMetaData.java#L102-L104]
 * 
[ColumnChunkMetaData.getStartingPos()|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184-L193]

As per my understanding the issue is based on the following to have the wrong 
offsets of {{rowGroup ~n~}} (where we have {{k}} columns):
* {{columnChunk ~n-1, 1~}} (first column chunk of {{rowGroup ~n-1~}}) is 
dictionary encoded as well as {{columnChunk ~n-1, k~}}
* {{columnChunk ~n, 1~}} is not dictionary encoded
In this case {{fileOffset ~n~ = dictionaryOffset ~n, 1~ = dictionaryOffset 
~n-1, k~}}

To discover this issue we should check if a column chunk is dictionary encoded 
before using the dictionary offset of it. Unfortunately, we have to do the same 
before using the file offset of a row group, or simply ignore this value and 
use the offsets of the first column chunk with the check.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
##
@@ -239,6 +248,82 @@ public void testWriteRead() throws Exception {
 PrintFooter.main(new String[] {path.toString()});
   }
 
+  @Test
+  public void testWriteReadWithRecordReader() throws Exception {

Review comment:
   I think this unit test does not reproduce the original scenario. Please 
check my comment in the 
[jira](https://issues.apache.org/jira/browse/PARQUET-2078?focusedCommentId=17406621=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17406621).




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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> 

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Nemon Lou (Jira)


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

Nemon Lou commented on PARQUET-2078:


The wrongly setted dictionary page offset does not spread 
from org.apache.parquet.hadoop.metadata.ColumnChunkMetaData 
to org.apache.parquet.format.ColumnMetaData
due to hasDictionaryPages check here:
https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L520

While rowGroup.file_offset still uses ColumnChunkMetaData to get starting point.

The test case added can pass after the patch and fail before patch, even if 
only modify the write path code.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> 

[GitHub] [parquet-mr] huaxingao commented on pull request #923: [PARQUET-1968] FilterApi support In predicate

2021-08-30 Thread GitBox


huaxingao commented on pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#issuecomment-908849008


   @gszadovszky @shangxinli @dbtsai Thank you all very much for reviewing! I 
have changed the code to generate the visit methods for in/notIn and also added 
the default by throwing Exception. Will address the rest of the comments 
tomorrow or the day after tomorrow. 


-- 
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-1968) FilterApi support In predicate

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1968:
-

huaxingao commented on pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#issuecomment-908849008


   @gszadovszky @shangxinli @dbtsai Thank you all very much for reviewing! I 
have changed the code to generate the visit methods for in/notIn and also added 
the default by throwing Exception. Will address the rest of the comments 
tomorrow or the day after tomorrow. 


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


> FilterApi support In predicate
> --
>
> Key: PARQUET-1968
> URL: https://issues.apache.org/jira/browse/PARQUET-1968
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Yuming Wang
>Priority: Major
>
> FilterApi should support native In predicate.
> Spark:
> https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala#L600-L605
> Impala:
> https://issues.apache.org/jira/browse/IMPALA-3654



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


[jira] [Commented] (PARQUET-1968) FilterApi support In predicate

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1968:
-

huaxingao commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698937308



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
##
@@ -123,6 +124,46 @@ public boolean accept(Visitor visitor) {
 }
   }
 
+  abstract class SetInspector implements IncrementallyUpdatedFilterPredicate {

Review comment:
   Changed. Thanks!

##
File path: pom.xml
##
@@ -478,6 +478,7 @@
 change to fix a integer overflow issue.
 TODO: remove this after Parquet 1.13 release -->
   
org.apache.parquet.column.values.dictionary.DictionaryValuesWriter#dictionaryByteSize
+  
org.apache.parquet.filter2.predicate.FilterPredicate

Review comment:
   Changed. Thanks!




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


> FilterApi support In predicate
> --
>
> Key: PARQUET-1968
> URL: https://issues.apache.org/jira/browse/PARQUET-1968
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Yuming Wang
>Priority: Major
>
> FilterApi should support native In predicate.
> Spark:
> https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala#L600-L605
> Impala:
> https://issues.apache.org/jira/browse/IMPALA-3654



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


[GitHub] [parquet-mr] huaxingao commented on a change in pull request #923: [PARQUET-1968] FilterApi support In predicate

2021-08-30 Thread GitBox


huaxingao commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698937308



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
##
@@ -123,6 +124,46 @@ public boolean accept(Visitor visitor) {
 }
   }
 
+  abstract class SetInspector implements IncrementallyUpdatedFilterPredicate {

Review comment:
   Changed. Thanks!

##
File path: pom.xml
##
@@ -478,6 +478,7 @@
 change to fix a integer overflow issue.
 TODO: remove this after Parquet 1.13 release -->
   
org.apache.parquet.column.values.dictionary.DictionaryValuesWriter#dictionaryByteSize
+  
org.apache.parquet.filter2.predicate.FilterPredicate

Review comment:
   Changed. Thanks!




-- 
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-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], you are right, so {{dictionaryPageOffset}} is not impacted. Great 
news. 

After the second look it is not required to have the first column being 
dictionary encoded before the invalid row group. It is enough that there are 
dictionary encoded column chunks in the previous row groups and that the first 
column chunk is not dictionary encoded in the invalid row group. So, [~nemon], 
you also right with your PR. 

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
##
@@ -239,6 +248,82 @@ public void testWriteRead() throws Exception {
 PrintFooter.main(new String[] {path.toString()});
   }
 
+  @Test
+  public void testWriteReadWithRecordReader() throws Exception {

Review comment:
   My bad, it seems just fine.




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




Re: Any Parquet implementations might be impacted by PARQUET-2078

2021-08-30 Thread Gabor Szadovszky
It turned out that ColumnMetaData.dictionary_page_offset is not impacted by
this issue so it is much easier to handle. It seems that 1.12.0 is the
first parquet_mr release which writes ColumnChunk.file_offset and according
to PARQUET-2078

it
is invalid in certain cases. So any implementations need to have a way to
calculate/gather this offset without using the actual field. What we need
to ensure is that no one relies on the value of ColumnChunk.file_offset at
least in cases when the file was written by parquet-mr 1.12.0.

I've also created PARQUET-2080
 to deprecate the field
in the format.

Regards,
Gabor

On Fri, Aug 27, 2021 at 11:11 AM Gabor Szadovszky  wrote:

> Hi everyone,
>
> It turned out that since parquet-mr 1.12.0 in certain conditions we write
> wrong values into ColumnMetaData.dictionary_page_offset
> 
>  and
> ColumnChunk.file_offset
> .
> See details in PARQUET-2078
> .
> Because of that any implementations that use these values have to be
> prepared for potential invalid values in case the file is written by
> parquet-mr 1.12.0.
>
> As per my understanding of the issue (to be verified) the
> distinguish between valid and invalid values of these offsets is quite
> easy: dictionary_page_offset is set to a value while the column chunk is
> not dictionary encoded (as per ColumnMetaData.encodings
> ).
> In this case we have to use the offset of the first data page
> 
> in the first column chunk of the row group.
>
> Regards,
> Gabor
>


[jira] [Commented] (PARQUET-2080) Deprecate RowGroup.file_offset

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2080:
-

gszadovszky opened a new pull request #178:
URL: https://github.com/apache/parquet-format/pull/178


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### 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.

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

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


> Deprecate RowGroup.file_offset
> --
>
> Key: PARQUET-2080
> URL: https://issues.apache.org/jira/browse/PARQUET-2080
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Due to PARQUET-2078 RowGroup.file_offset is not reliable. We shall deprecate 
> the field and add suggestions how to calculate the value.



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


[GitHub] [parquet-format] gszadovszky opened a new pull request #178: PARQUET-2080: Deprecate RowGroup.file_offset

2021-08-30 Thread GitBox


gszadovszky opened a new pull request #178:
URL: https://github.com/apache/parquet-format/pull/178


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### 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.

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-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
##
@@ -239,6 +248,82 @@ public void testWriteRead() throws Exception {
 PrintFooter.main(new String[] {path.toString()});
   }
 
+  @Test
+  public void testWriteReadWithRecordReader() throws Exception {

Review comment:
   My bad, it seems just fine.




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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  

[jira] [Assigned] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2078:
-

Assignee: Nemon Lou

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method) ~[?:1.8.0_292]
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>  ~[?:1.8.0_292]
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>  ~[?:1.8.0_292]
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423) 
> 

[jira] [Created] (PARQUET-2080) Deprecate RowGroup.file_offset

2021-08-30 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2080:
-

 Summary: Deprecate RowGroup.file_offset
 Key: PARQUET-2080
 URL: https://issues.apache.org/jira/browse/PARQUET-2080
 Project: Parquet
  Issue Type: Bug
  Components: parquet-format
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


Due to PARQUET-2078 RowGroup.file_offset is not reliable. We shall deprecate 
the field and add suggestions how to calculate the value.



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


[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

shangxinli commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908428874


   @ggershinsky Do you want to have a look? 


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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> 

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

gszadovszky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908452452


   @ggershinsky, even though this PR fixes the write path as well we have 
already released 1.12.0 so we have to prepare for the case of 
`RowGroup.file_offset` is incorrect. 


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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  

[GitHub] [parquet-mr] gszadovszky commented on pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


gszadovszky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908452452


   @ggershinsky, even though this PR fixes the write path as well we have 
already released 1.12.0 so we have to prepare for the case of 
`RowGroup.file_offset` is incorrect. 


-- 
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-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

ggershinsky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908463834


   Yep, but the current fix perpetuates the situation where some readers can't 
process encrypted files, even if they have keys for all projected columns; 
doesn't look like an optimal long-term solution.. I'm just back online after a 
vacation; will go over the details in this thread, maybe something else can be 
done 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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  

[GitHub] [parquet-mr] ggershinsky commented on pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


ggershinsky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908463834


   Yep, but the current fix perpetuates the situation where some readers can't 
process encrypted files, even if they have keys for all projected columns; 
doesn't look like an optimal long-term solution.. I'm just back online after a 
vacation; will go over the details in this thread, maybe something else can be 
done 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-1968) FilterApi support In predicate

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1968:
-

shangxinli commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698606499



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);

Review comment:
   I see you have a 'toString' to cache but do we see generally this is 
reused multiple times? If no, proactively converting to string will be a waste. 
 

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);
+  StringBuilder str = new StringBuilder();
+  int iter = 0;
+  for (T value : values) {
+if (iter >= 100) break;
+str.append(value).append(", ");
+iter++;
+  }
+  String valueStr = values.size() <= 100 ? str.substring(0, str.length() - 
2) : str + "...";
+  this.toString = name + "(" + column.getColumnPath().toDotString() + ", " 
+ valueStr + ")";

Review comment:
   Would it be possible to merge lines 272 and 273 into the above code of 
that building? the string? String operations sometimes consume a lot of memory 
like this. 

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);
+  StringBuilder str = new StringBuilder();
+  int iter = 0;
+  for (T value : values) {
+if (iter >= 100) break;
+str.append(value).append(", ");
+iter++;
+  }
+  String valueStr = values.size() <= 100 ? str.substring(0, str.length() - 
2) : str + "...";
+  this.toString = name + "(" + column.getColumnPath().toDotString() + ", " 
+ valueStr + ")";
+}
+
+public Column getColumn() {
+  return column;
+}
+
+public Set getValues() {
+  return values;
+}
+
+@Override
+public String toString() {
+  return toString;
+}
+
+@Override
+public boolean equals(Object o) {
+  if (this == o) return true;
+  if (o == null || getClass() != o.getClass()) return false;

Review comment:
   I guess you can just 'return this.getClass() == o.getClass()'

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be 

[GitHub] [parquet-mr] shangxinli commented on pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


shangxinli commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908428874


   @ggershinsky Do you want to have a look? 


-- 
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-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2078:
-

ggershinsky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908443796


   Sure. This won't work if the first column is encrypted and the reader 
doesn't have its key. Can the "write" part be fixed instead, so the RowGroup 
offset is set correctly?


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


> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  

[GitHub] [parquet-mr] ggershinsky commented on pull request #925: PARQUET-2078: Failed to read parquet file after writing with the same …

2021-08-30 Thread GitBox


ggershinsky commented on pull request #925:
URL: https://github.com/apache/parquet-mr/pull/925#issuecomment-908443796


   Sure. This won't work if the first column is encrypted and the reader 
doesn't have its key. Can the "write" part be fixed instead, so the RowGroup 
offset is set correctly?


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

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

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




[GitHub] [parquet-mr] shangxinli commented on a change in pull request #923: [PARQUET-1968] FilterApi support In predicate

2021-08-30 Thread GitBox


shangxinli commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698598441



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn

Review comment:
   Have a better comment since it is public method  




-- 
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-1968) FilterApi support In predicate

2021-08-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1968:
-

shangxinli commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698598441



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn

Review comment:
   Have a better comment since it is public method  




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


> FilterApi support In predicate
> --
>
> Key: PARQUET-1968
> URL: https://issues.apache.org/jira/browse/PARQUET-1968
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Yuming Wang
>Priority: Major
>
> FilterApi should support native In predicate.
> Spark:
> https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala#L600-L605
> Impala:
> https://issues.apache.org/jira/browse/IMPALA-3654



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


[GitHub] [parquet-mr] shangxinli commented on a change in pull request #923: [PARQUET-1968] FilterApi support In predicate

2021-08-30 Thread GitBox


shangxinli commented on a change in pull request #923:
URL: https://github.com/apache/parquet-mr/pull/923#discussion_r698606499



##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);

Review comment:
   I see you have a 'toString' to cache but do we see generally this is 
reused multiple times? If no, proactively converting to string will be a waste. 
 

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);
+  StringBuilder str = new StringBuilder();
+  int iter = 0;
+  for (T value : values) {
+if (iter >= 100) break;
+str.append(value).append(", ");
+iter++;
+  }
+  String valueStr = values.size() <= 100 ? str.substring(0, str.length() - 
2) : str + "...";
+  this.toString = name + "(" + column.getColumnPath().toDotString() + ", " 
+ valueStr + ")";

Review comment:
   Would it be possible to merge lines 272 and 273 into the above code of 
that building? the string? String operations sometimes consume a lot of memory 
like this. 

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);
+  StringBuilder str = new StringBuilder();
+  int iter = 0;
+  for (T value : values) {
+if (iter >= 100) break;
+str.append(value).append(", ");
+iter++;
+  }
+  String valueStr = values.size() <= 100 ? str.substring(0, str.length() - 
2) : str + "...";
+  this.toString = name + "(" + column.getColumnPath().toDotString() + ", " 
+ valueStr + ")";
+}
+
+public Column getColumn() {
+  return column;
+}
+
+public Set getValues() {
+  return values;
+}
+
+@Override
+public String toString() {
+  return toString;
+}
+
+@Override
+public boolean equals(Object o) {
+  if (this == o) return true;
+  if (o == null || getClass() != o.getClass()) return false;

Review comment:
   I guess you can just 'return this.getClass() == o.getClass()'

##
File path: 
parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
##
@@ -247,6 +250,80 @@ public int hashCode() {
 }
   }
 
+  // base class for In and NotIn
+  public static abstract class SetColumnFilterPredicate> implements FilterPredicate, Serializable {
+private final Column column;
+private final Set values;
+private final String toString;
+
+protected SetColumnFilterPredicate(Column column, Set values) {
+  this.column = Objects.requireNonNull(column, "column cannot be null");
+  this.values = Objects.requireNonNull(values, "values cannot be null");
+  checkArgument(!values.isEmpty(), "values in SetColumnFilterPredicate 
shouldn't be empty!");
+
+  String name = 

[jira] [Created] (PARQUET-2081) Encryption translation tool - Parquet-hadoop

2021-08-30 Thread Xinli Shang (Jira)
Xinli Shang created PARQUET-2081:


 Summary: Encryption translation tool - Parquet-hadoop
 Key: PARQUET-2081
 URL: https://issues.apache.org/jira/browse/PARQUET-2081
 Project: Parquet
  Issue Type: Task
  Components: parquet-mr
Reporter: Xinli Shang
 Fix For: 1.13.0


This is the implement the core part of the Encryption translation tool in 
parquet-hadoop. After this, we will have another Jira/PR for parquet-cli to 
integrate with key tools for encryption properties.. 



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


[jira] [Created] (PARQUET-2082) Encryption translation tool - Parquet-cli

2021-08-30 Thread Xinli Shang (Jira)
Xinli Shang created PARQUET-2082:


 Summary: Encryption translation tool - Parquet-cli
 Key: PARQUET-2082
 URL: https://issues.apache.org/jira/browse/PARQUET-2082
 Project: Parquet
  Issue Type: Task
Reporter: Xinli Shang


This is to implement the parquet-cli part of the encryption translation tool. 
It integrates with key tools to build the encryption properties, handle the 
parameters and call the parquet-hadoop API to encrypt. 



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