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

ASF GitHub Bot commented on PARQUET-1982:
-----------------------------------------

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



##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##########
@@ -920,22 +946,52 @@ public PageReadStore readNextRowGroup() throws 
IOException {
       }
     }
     // actually read all the chunks
-    ChunkListBuilder builder = new ChunkListBuilder();
+    ChunkListBuilder builder = new ChunkListBuilder(block.getRowCount());
     for (ConsecutivePartList consecutiveChunks : allParts) {
       consecutiveChunks.readAll(f, builder);
     }
     for (Chunk chunk : builder.build()) {
-      readChunkPages(chunk, block);
+      readChunkPages(chunk, block, rowGroup);
     }
 
-    // avoid re-reading bytes the dictionary reader is used after this call
-    if (nextDictionaryReader != null) {
-      nextDictionaryReader.setRowGroup(currentRowGroup);
+    return rowGroup;
+  }
+
+  /**
+   * Reads all the columns requested from the specified row group. It may skip 
specific pages based on the column
+   * indexes according to the actual filter. As the rows are not aligned among 
the pages of the different columns row
+   * synchronization might be required. See the documentation of the class 
SynchronizingColumnReader for details.
+   *
+   * @param blockIndex the index of the requested block
+   * @return the PageReadStore which can provide PageReaders for each column 
or null if there are no rows in this block
+   * @throws IOException if an error occurs while reading
+   */
+  public PageReadStore readFilteredRowGroup(int blockIndex) throws IOException 
{

Review comment:
       It depends on how do you plan to use it. If you don't plan to cache the 
related metadata outside of the reader I think it is more clean to use the 
index because it does not suggest that you may use any arbitrary metadata but 
the ones are in the file. Meanwhile, you may want to select specific row groups 
to be read based on the metadata so you would already have the related object. 
In this case it is easier to simply pass it instead of checking for the index.




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

For queries about this service, please contact Infrastructure at:
[email protected]


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



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

Reply via email to