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

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

fschmalzel commented on pull request #871:
URL: https://github.com/apache/parquet-mr/pull/871#issuecomment-818724064


   Tested random access with a 351 MB parquet file with zstd and dictionary 
enabled. The file has 321 row groups.
   Using the sequential method to read the randomized indexes took around 87319 
milliseconds.
   Using the sequential method without this commit took around 86311 
milliseconds.
   Using the random access method took around 58941 milliseconds.
   
   Note: This only improves randomized access. For sequential access both 
methods should take about the same time. Nevertheless the sequential method is 
not removed, so that should not be a problem.
   
   This should all be taken with a grain of salt, as this was tested without 
something like jmh. 
https://github.com/openjdk/jmh#java-microbenchmark-harness-jmh
   
   Unfortunately i cannot upload the tested file somewhere. If you want me to 
test a different file i will gladly do so.
   
   Quick and dirty test:
   
   ```java
   package org.apache.parquet;
   
   import org.apache.hadoop.conf.Configuration;
   import org.apache.hadoop.fs.Path;
   import org.apache.parquet.column.page.PageReadStore;
   import org.apache.parquet.example.data.Group;
   import org.apache.parquet.example.data.simple.convert.GroupRecordConverter;
   import org.apache.parquet.hadoop.ParquetFileReader;
   import org.apache.parquet.hadoop.util.HadoopInputFile;
   import org.apache.parquet.io.ColumnIOFactory;
   import org.apache.parquet.io.MessageColumnIO;
   import org.apache.parquet.io.RecordReader;
   import org.apache.parquet.schema.MessageType;
   import org.junit.Test;
   
   import java.io.IOException;
   import java.util.ArrayList;
   import java.util.Collections;
   import java.util.List;
   import java.util.Random;
   
   public class ReadTest {
   
     private static final long RANDOM_SEED = 7174252115631550700L;
     private static final String FILE_PATH = 
"C:\\Users\\fes\\Desktop\\test.parquet";
     private static final int ROUNDS = 4;
   
     @Test
     public void testSequential() throws IOException {
       Random random = new Random(RANDOM_SEED);
   
       Path file = new Path(FILE_PATH);
       Configuration configuration = new Configuration();
       ParquetReadOptions options = ParquetReadOptions.builder().build();
   
       ParquetFileReader reader = new 
ParquetFileReader(HadoopInputFile.fromPath(file, configuration), options);
       MessageType schema = reader.getFileMetaData().getSchema();
       MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
       GroupRecordConverter converter = new GroupRecordConverter(schema);
   
       int blockAmount = reader.getRowGroups().size();
   
       List<Integer> indexes = new ArrayList<>();
       for (int j = 0; j < ROUNDS; j++) {
         for (int i = 0; i < blockAmount; i++) {
           indexes.add(i);
         }
       }
   
       Collections.shuffle(indexes, random);
   
       long start = System.currentTimeMillis();
   
       int currentRowGroup = -1;
   
       for (int index : indexes) {
         if (index <= currentRowGroup) {
           try {
             reader.close();
           } catch (Exception ignored) {
           }
   
           reader = new ParquetFileReader(HadoopInputFile.fromPath(file, 
configuration), options);
           currentRowGroup = -1;
         }
         for (int i = 1; i < index - currentRowGroup; i++) {
           reader.skipNextRowGroup();
         }
         currentRowGroup = index;
         PageReadStore pages = reader.readNextRowGroup();
         long rowCount = pages.getRowCount();
         RecordReader<Group> recordReader = columnIO.getRecordReader(pages, 
converter);
         for (long i = 0; i < rowCount; i++) {
           recordReader.read();
         }
       }
   
       long stop = System.currentTimeMillis();
   
       try {
         reader.close();
       } catch (Exception ignored) {
       }
   
       long timeTaken = stop - start;
   
       System.out.printf("Sequential access took %d milliseconds%n", timeTaken);
     }
   
     @Test
     public void testRandom() throws IOException {
       Random random = new Random(RANDOM_SEED);
   
       Path file = new Path(FILE_PATH);
       Configuration configuration = new Configuration();
       ParquetReadOptions options = ParquetReadOptions.builder().build();
   
       ParquetFileReader reader = new 
ParquetFileReader(HadoopInputFile.fromPath(file, configuration), options);
       MessageType schema = reader.getFileMetaData().getSchema();
       MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
       GroupRecordConverter converter = new GroupRecordConverter(schema);
   
       int blockAmount = reader.getRowGroups().size();
   
       List<Integer> indexes = new ArrayList<>();
       for (int j = 0; j < ROUNDS; j++) {
         for (int i = 0; i < blockAmount; i++) {
           indexes.add(i);
         }
       }
   
       Collections.shuffle(indexes, random);
   
       long start = System.currentTimeMillis();
       for (int index : indexes) {
         PageReadStore pages = reader.readRowGroup(index);
         long rowCount = pages.getRowCount();
         RecordReader<Group> recordReader = columnIO.getRecordReader(pages, 
converter);
         for (long i = 0; i < rowCount; i++) {
           recordReader.read();
         }
       }
   
       long stop = System.currentTimeMillis();
   
       try {
         reader.close();
       } catch (Exception ignored) {
       }
   
       long timeTaken = stop - start;
   
       System.out.printf("Random access took %d milliseconds%n", timeTaken);
     }
     
   }
   ```
   


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