You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "ConeyLiu (via GitHub)" <gi...@apache.org> on 2023/11/03 09:22:12 UTC

[PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

ConeyLiu opened a new pull request, #1183:
URL: https://github.com/apache/parquet-mr/pull/1183

   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).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   ### 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 patch aims to reduce the unnecessary RowGroup data reading during rewriting.
   


-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#issuecomment-1792112960

   @wgtmac A small improvement, please take a look when you are free. Thanks a lot.


-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#discussion_r1382413034


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -265,14 +265,9 @@ public void processBlocks() throws IOException {
   }
 
   private void processBlocksFromReader(IndexCache indexCache) throws IOException {
-    PageReadStore store = reader.readNextRowGroup();
-    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, originalCreatedBy);
-
-    int blockId = 0;
-    while (store != null) {
-      writer.startBlock(store.getRowCount());
-
+    for (int blockId = 0; blockId < meta.getBlocks().size(); blockId ++) {
       BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      writer.startBlock(blockMetaData.getRowCount());

Review Comment:
   Make sense!



-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on code in PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#discussion_r1381382668


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -265,14 +265,9 @@ public void processBlocks() throws IOException {
   }
 
   private void processBlocksFromReader(IndexCache indexCache) throws IOException {
-    PageReadStore store = reader.readNextRowGroup();
-    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, originalCreatedBy);
-
-    int blockId = 0;
-    while (store != null) {
-      writer.startBlock(store.getRowCount());

Review Comment:
   We don't need to get the row count from the `PageReadStore` since getting the `PageReadStore` will force read the whole row group data into memory.



-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on code in PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#discussion_r1381384113


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -323,7 +318,7 @@ private void processBlocksFromReader(IndexCache indexCache) throws IOException {
           }
 
           // Translate compression and/or encryption
-          writer.startColumn(descriptor, crStore.getColumnReader(descriptor).getTotalValueCount(), newCodecName);
+          writer.startColumn(descriptor, chunk.getValueCount(), newCodecName);

Review Comment:
   Same here, just get the value count from the ColumnChunkMetaData



-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac merged PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183


-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#issuecomment-1803078643

   Thanks @wgtmac 


-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on code in PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#discussion_r1381383424


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -265,14 +265,9 @@ public void processBlocks() throws IOException {
   }
 
   private void processBlocksFromReader(IndexCache indexCache) throws IOException {
-    PageReadStore store = reader.readNextRowGroup();
-    ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new DummyGroupConverter(), schema, originalCreatedBy);
-
-    int blockId = 0;
-    while (store != null) {
-      writer.startBlock(store.getRowCount());
-
+    for (int blockId = 0; blockId < meta.getBlocks().size(); blockId ++) {
       BlockMetaData blockMetaData = meta.getBlocks().get(blockId);
+      writer.startBlock(blockMetaData.getRowCount());

Review Comment:
   Just get the row count from the `BlockMetaData`



-- 
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-unsubscribe@parquet.apache.org

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


Re: [PR] PARQUET-2372: Avoid unnecessary reading of RowGroup data during rewriting [parquet-mr]

Posted by "ConeyLiu (via GitHub)" <gi...@apache.org>.
ConeyLiu commented on code in PR #1183:
URL: https://github.com/apache/parquet-mr/pull/1183#discussion_r1381384663


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java:
##########
@@ -666,6 +658,8 @@ private void nullifyColumn(ColumnDescriptor descriptor,
 
     long totalChunkValues = chunk.getValueCount();
     int dMax = descriptor.getMaxDefinitionLevel();
+    PageReadStore pageReadStore = reader.readRowGroup(blockIndex);

Review Comment:
   Only get the `PageReadStore` when needed.



-- 
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-unsubscribe@parquet.apache.org

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