[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066592694


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Added the file path to the log. Please take a look again. Thanks!





> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-10 Thread GitBox


wgtmac commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066592694


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Added the file path to the log. Please take a look again. 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-2160) Close decompression stream to free off-heap memory in time

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2160:
-

camper42 commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1378181577

   same problem with @alexeykudinkin 
   
   currently we replace paruqet jar with patched one in our image, waiting for 
release




> Close decompression stream to free off-heap memory in time
> --
>
> Key: PARQUET-2160
> URL: https://issues.apache.org/jira/browse/PARQUET-2160
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Affects Versions: 1.12.3
> Environment: Spark 3.1.2 + Iceberg 0.12 + Parquet 1.12.3 + zstd-jni 
> 1.4.9.1 + glibc
>Reporter: Yujiang Zhong
>Priority: Blocker
>
> The decompressed stream in HeapBytesDecompressor$decompress now relies on the 
> JVM GC to close. When reading parquet in zstd compressed format, sometimes I 
> ran into OOM cause high off-heap usage. I think the reason is that the GC is 
> not timely and causes off-heap memory fragmentation. I had to set  lower 
> MALLOC_TRIM_THRESHOLD_ to make glibc give back memory to system quickly. 
> There is a 
> [thread|[https://apache-iceberg.slack.com/archives/C025PH0G1D4/p1650928750269869?thread_ts=1650927062.590789=C025PH0G1D4]]
>  of this zstd parquet issus in Iceberg community slack:  some people had the 
> same problem. 
> I think maybe we can use ByteArrayBytesInput as decompressed bytes input and 
> close decompressed stream in time to solve this problem:
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.from(is, uncompressedSize); {code}
> ->
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.copy(BytesInput.from(is, uncompressedSize));
> is.close(); {code}
> After I made this change to decompress, I found off-heap memory is 
> significantly reduced (with same query on same data).



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


[GitHub] [parquet-mr] camper42 commented on pull request #982: PARQUET-2160: Close ZstdInputStream to free off-heap memory in time.

2023-01-10 Thread GitBox


camper42 commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1378181577

   same problem with @alexeykudinkin 
   
   currently we replace paruqet jar with patched one in our image, waiting for 
release


-- 
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-2075) Unified Rewriter Tool

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377840323

   Thanks a lot @gszadovszky 




> Unified Rewriter Tool  
> ---
>
> Key: PARQUET-2075
> URL: https://issues.apache.org/jira/browse/PARQUET-2075
> Project: Parquet
>  Issue Type: New Feature
>Reporter: Xinli Shang
>Assignee: Gang Wu
>Priority: Major
>
> During the discussion of PARQUET-2071, we came up with the idea of a 
> universal tool to translate the existing file to a different state while 
> skipping some level steps like encoding/decoding, to gain speed. For example, 
> only decompress pages and then compress directly. For PARQUET-2071, we only 
> decrypt and then encrypt directly. This will be useful for the existing data 
> to onboard Parquet features like column encryption, zstd etc. 
> We already have tools like trans-compression, column pruning etc. We will 
> consolidate all these tools with this universal tool. 



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


[GitHub] [parquet-mr] shangxinli commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-10 Thread GitBox


shangxinli commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377840323

   Thanks a lot @gszadovszky 


-- 
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-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-10 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1980:
---

Perfect. Thank you, [~mgrigorov]!

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



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


[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377706622

   > @gszadovszky I Just want to check if you have time to have a look. @wgtmac 
just be nice to take over the work that we discussed earlier to have an 
aggregated rewriter.
   
   @shangxinli, I'll try to take a look this week.




> Unified Rewriter Tool  
> ---
>
> Key: PARQUET-2075
> URL: https://issues.apache.org/jira/browse/PARQUET-2075
> Project: Parquet
>  Issue Type: New Feature
>Reporter: Xinli Shang
>Assignee: Gang Wu
>Priority: Major
>
> During the discussion of PARQUET-2071, we came up with the idea of a 
> universal tool to translate the existing file to a different state while 
> skipping some level steps like encoding/decoding, to gain speed. For example, 
> only decompress pages and then compress directly. For PARQUET-2071, we only 
> decrypt and then encrypt directly. This will be useful for the existing data 
> to onboard Parquet features like column encryption, zstd etc. 
> We already have tools like trans-compression, column pruning etc. We will 
> consolidate all these tools with this universal tool. 



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-10 Thread GitBox


gszadovszky commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1377706622

   > @gszadovszky I Just want to check if you have time to have a look. @wgtmac 
just be nice to take over the work that we discussed earlier to have an 
aggregated rewriter.
   
   @shangxinli, I'll try to take a look this week.


-- 
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-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

gszadovszky commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377700950

   > @gszadovszky Nice to see you are back!
   
   @shangxinli, I wouldn't say I'm back, unfortunately. I'm a bit closer to 
Parquet at Dremio but actually not working on it. We'll see if I will have some 
spare time for it. :)




> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-10 Thread GitBox


gszadovszky commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377700950

   > @gszadovszky Nice to see you are back!
   
   @shangxinli, I wouldn't say I'm back, unfortunately. I'm a bit closer to 
Parquet at Dremio but actually not working on it. We'll see if I will have some 
spare time for it. :)


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

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

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



[jira] [Updated] (PARQUET-2225) [C++] Allow reading dense with RecordReader

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated PARQUET-2225:

Labels: pull-request-available  (was: )

> [C++] Allow reading dense with RecordReader
> ---
>
> Key: PARQUET-2225
> URL: https://issues.apache.org/jira/browse/PARQUET-2225
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-cpp
>Reporter: fatemah
>Assignee: fatemah
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently ReadRecords reads spaced by default. Some readers may need to read 
> the values dense, and reading spaced is less efficient than reading dense. We 
> need an option for reading dense.



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


[jira] [Updated] (PARQUET-2225) [C++] Allow reading dense with RecordReader

2023-01-10 Thread fatemah (Jira)


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

fatemah updated PARQUET-2225:
-
Summary: [C++] Allow reading dense with RecordReader  (was: Allow reading 
dense with RecordReader)

> [C++] Allow reading dense with RecordReader
> ---
>
> Key: PARQUET-2225
> URL: https://issues.apache.org/jira/browse/PARQUET-2225
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-cpp
>Reporter: fatemah
>Assignee: fatemah
>Priority: Major
>
> Currently ReadRecords reads spaced by default. Some readers may need to read 
> the values dense, and reading spaced is less efficient than reading dense. We 
> need an option for reading dense.



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


[jira] [Created] (PARQUET-2225) Allow reading dense with RecordReader

2023-01-10 Thread fatemah (Jira)
fatemah created PARQUET-2225:


 Summary: Allow reading dense with RecordReader
 Key: PARQUET-2225
 URL: https://issues.apache.org/jira/browse/PARQUET-2225
 Project: Parquet
  Issue Type: New Feature
  Components: parquet-cpp
Reporter: fatemah
Assignee: fatemah


Currently ReadRecords reads spaced by default. Some readers may need to read 
the values dense, and reading spaced is less efficient than reading dense. We 
need an option for reading dense.



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


[jira] [Commented] (PARQUET-2160) Close decompression stream to free off-heap memory in time

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2160:
-

shangxinli commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1377598327

   Thanks @alexeykudinkin for the explanation. 




> Close decompression stream to free off-heap memory in time
> --
>
> Key: PARQUET-2160
> URL: https://issues.apache.org/jira/browse/PARQUET-2160
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Affects Versions: 1.12.3
> Environment: Spark 3.1.2 + Iceberg 0.12 + Parquet 1.12.3 + zstd-jni 
> 1.4.9.1 + glibc
>Reporter: Yujiang Zhong
>Priority: Blocker
>
> The decompressed stream in HeapBytesDecompressor$decompress now relies on the 
> JVM GC to close. When reading parquet in zstd compressed format, sometimes I 
> ran into OOM cause high off-heap usage. I think the reason is that the GC is 
> not timely and causes off-heap memory fragmentation. I had to set  lower 
> MALLOC_TRIM_THRESHOLD_ to make glibc give back memory to system quickly. 
> There is a 
> [thread|[https://apache-iceberg.slack.com/archives/C025PH0G1D4/p1650928750269869?thread_ts=1650927062.590789=C025PH0G1D4]]
>  of this zstd parquet issus in Iceberg community slack:  some people had the 
> same problem. 
> I think maybe we can use ByteArrayBytesInput as decompressed bytes input and 
> close decompressed stream in time to solve this problem:
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.from(is, uncompressedSize); {code}
> ->
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.copy(BytesInput.from(is, uncompressedSize));
> is.close(); {code}
> After I made this change to decompress, I found off-heap memory is 
> significantly reduced (with same query on same data).



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


[GitHub] [parquet-mr] shangxinli commented on pull request #982: PARQUET-2160: Close ZstdInputStream to free off-heap memory in time.

2023-01-10 Thread GitBox


shangxinli commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1377598327

   Thanks @alexeykudinkin for the explanation. 


-- 
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-2160) Close decompression stream to free off-heap memory in time

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2160:
-

alexeykudinkin commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1377589036

   Totally @shangxinli  
   
   We have running Spark clusters in production _ingesting_ from 100s of Apache 
Hudi tables (using Parquet and Zstd) and writing into other ones. We switched 
from gzip to zstd slightly over a month ago and we started to have OOM issues 
almost immediately. It took us a bit of triaging to zero in on zstd, but now 
we're confident that it's not mis-calibration of our configs but slow-bleeding 
leak of the native memory.
   
   The crux of the problem is very particular type of the job 

> Close decompression stream to free off-heap memory in time
> --
>
> Key: PARQUET-2160
> URL: https://issues.apache.org/jira/browse/PARQUET-2160
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Affects Versions: 1.12.3
> Environment: Spark 3.1.2 + Iceberg 0.12 + Parquet 1.12.3 + zstd-jni 
> 1.4.9.1 + glibc
>Reporter: Yujiang Zhong
>Priority: Blocker
>
> The decompressed stream in HeapBytesDecompressor$decompress now relies on the 
> JVM GC to close. When reading parquet in zstd compressed format, sometimes I 
> ran into OOM cause high off-heap usage. I think the reason is that the GC is 
> not timely and causes off-heap memory fragmentation. I had to set  lower 
> MALLOC_TRIM_THRESHOLD_ to make glibc give back memory to system quickly. 
> There is a 
> [thread|[https://apache-iceberg.slack.com/archives/C025PH0G1D4/p1650928750269869?thread_ts=1650927062.590789=C025PH0G1D4]]
>  of this zstd parquet issus in Iceberg community slack:  some people had the 
> same problem. 
> I think maybe we can use ByteArrayBytesInput as decompressed bytes input and 
> close decompressed stream in time to solve this problem:
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.from(is, uncompressedSize); {code}
> ->
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.copy(BytesInput.from(is, uncompressedSize));
> is.close(); {code}
> After I made this change to decompress, I found off-heap memory is 
> significantly reduced (with same query on same data).



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


[GitHub] [parquet-mr] alexeykudinkin commented on pull request #982: PARQUET-2160: Close ZstdInputStream to free off-heap memory in time.

2023-01-10 Thread GitBox


alexeykudinkin commented on PR #982:
URL: https://github.com/apache/parquet-mr/pull/982#issuecomment-1377589036

   Totally @shangxinli  
   
   We have running Spark clusters in production _ingesting_ from 100s of Apache 
Hudi tables (using Parquet and Zstd) and writing into other ones. We switched 
from gzip to zstd slightly over a month ago and we started to have OOM issues 
almost immediately. It took us a bit of triaging to zero in on zstd, but now 
we're confident that it's not mis-calibration of our configs but slow-bleeding 
leak of the native memory.
   
   The crux of the problem is very particular type of the job -- one that reads 
a lot of Zstd compressed Parquet (therefore triggering the affected path). 
Other jobs not reading Parquet are not affected.


-- 
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-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

shangxinli commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066042941


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -1038,7 +1044,10 @@ public PageReadStore readNextFilteredRowGroup() throws 
IOException {
 }
 BlockMetaData block = blocks.get(currentBlock);
 if (block.getRowCount() == 0L) {
-  throw new RuntimeException("Illegal row group of 0 rows");
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Same comments as above





> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-10 Thread GitBox


shangxinli commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066042941


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -1038,7 +1044,10 @@ public PageReadStore readNextFilteredRowGroup() throws 
IOException {
 }
 BlockMetaData block = blocks.get(currentBlock);
 if (block.getRowCount() == 0L) {
-  throw new RuntimeException("Illegal row group of 0 rows");
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Same comments as above



-- 
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-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

shangxinli commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066038932


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Any way to add file path?





> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-10 Thread GitBox


shangxinli commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1066038932


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Any way to add file path?



-- 
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-2075) Unified Rewriter Tool

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune and 

[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune and 

[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-10 Thread GitBox


wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune and mask same column");
+  }
+}
+
+if (encryptColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+

[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-10 Thread GitBox


wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1065962705


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune and mask same column");
+  }
+}
+
+if (encryptColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+

[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377485663

   > Thanks you for fixing this. I've added some comments. Also, could you add 
a similar test for the filtered row groups?
   
   Thanks for your review @gszadovszky !
   
   I have addressed all of your comments. Please take a look again.




> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-10 Thread GitBox


wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377485663

   > Thanks you for fixing this. I've added some comments. Also, could you add 
a similar test for the filtered row groups?
   
   Thanks for your review @gszadovszky !
   
   I have addressed all of your comments. Please take a look again.


-- 
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-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-10 Thread Martin Tzvetanov Grigorov (Jira)


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

Martin Tzvetanov Grigorov commented on PARQUET-1980:


Apache Arrow team is going to use self-hosted runners 
([https://lists.apache.org/thread/mskpqwpdq65t1wpj4f5klfq9217ljodw)] and this 
will allow to run on native hardware.

I guess later this year the same setup (Kubernetes based ephemeral runners) 
could be used by Apache Infra for more Apache projects, If this happens I will 
send a new PR!

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



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


[jira] [Commented] (PARQUET-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-10 Thread Martin Tzvetanov Grigorov (Jira)


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

Martin Tzvetanov Grigorov commented on PARQUET-1980:


Unfortunately the build takes more than 6 hours and Github Actions kills the CI 
workflow: [https://github.com/martin-g/parquet-mr/actions/runs/3873846602]

Here are the changes to migrate the workflow from TravisCI to Github Actions: 
[https://github.com/martin-g/parquet-mr/pull/1/files]

The x86_64 job is commented out just to disable it during the experimentation!

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



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


[jira] [Commented] (PARQUET-2212) Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2212:
-

wgtmac commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1376928785

   > @wgtmac Do you have time to have a look?
   
   @shangxinli Thanks for mentioning me. Sure, I will take a look this week.




> Add ByteBuffer api for decryptors to allow direct memory to be decrypted
> 
>
> Key: PARQUET-2212
> URL: https://issues.apache.org/jira/browse/PARQUET-2212
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Parth Chandra
>Priority: Major
> Fix For: 1.12.3
>
>
> The decrypt API in BlockCipher.Decryptor currently only provides an api that 
> takes in a byte array
> {code:java}
> byte[] decrypt(byte[] lengthAndCiphertext, byte[] AAD);{code}
> A parquet reader that uses the DirectByteBufferAllocator has to incur the 
> cost of copying the data into a byte array (and sometimes back to a 
> DirectByteBuffer) to decrypt data.
> This proposes adding a new API that accepts ByteBuffer as input and avoids 
> the data copy.
> {code:java}
> ByteBuffer decrypt(ByteBuffer from, byte[] AAD);{code}
> The decryption in ColumnChunkPageReadStore can also be updated to use the 
> ByteBuffer based api if the buffer is a DirectByteBuffer. If the buffer is a 
> HeapByteBuffer, then we can continue to use the byte array API since that 
> does not incur a copy when the underlying byte array is accessed.
> Also, some investigation has shown that decryption with ByteBuffers is not 
> able to use hardware acceleration in JVM's before JDK17. In those cases, the 
> overall decryption speed is faster with byte arrays even after incurring the 
> overhead of making a copy. 
> The proposal, then, is to enable the use of the ByteBuffer api for 
> DirectByteBuffers only, and only if the JDK is JDK17 or higher or the user 
> explicitly configures it. 



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


[GitHub] [parquet-mr] wgtmac commented on pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2023-01-10 Thread GitBox


wgtmac commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1376928785

   > @wgtmac Do you have time to have a look?
   
   @shangxinli Thanks for mentioning me. Sure, I will take a look this week.


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