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

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_r580349408



##########
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:
       What is the reason behind you are using a `BlockMetaData` argument for 
`readRowGroup` while you use the index of the row group here? I think, both 
should work similarly.

##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.*;
+import org.apache.parquet.statistics.DataGenerationContext;
+import org.apache.parquet.statistics.RandomValues;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {

Review comment:
       As far as I could understand you only check the first page of the same 
column for every row groups. I am not sure if it is enough or not but in this 
case the other columns are completely useless. I think, at least all the pages 
of the same column should be checked.
   
   Please, also test `readFilteredRowGroup`. As you do not decode the pages you 
do not need to test anything differently but we need to cover all new paths.

##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.*;
+import org.apache.parquet.statistics.DataGenerationContext;
+import org.apache.parquet.statistics.RandomValues;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {
+  private static final int KILOBYTE = 1 << 10;
+  private static final long RANDOM_SEED = 7174252115631550700L;
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  @Test
+  public void test() throws IOException {
+    Random random = new Random(RANDOM_SEED);
+
+    File file = temp.newFile("test_file.parquet");
+    file.delete();
+
+    int blockSize = 500 * KILOBYTE;
+    int pageSize = 20 * KILOBYTE;
+
+    List<DataContext> contexts = new ArrayList<>();
+
+    for (boolean enableDictionary : new boolean[]{false, true}) {
+      for (WriterVersion writerVersion : new 
WriterVersion[]{WriterVersion.PARQUET_1_0, WriterVersion.PARQUET_2_0}) {

Review comment:
       Of course it is not a problem but I think with/without dictionary and 
V1/V2 pages are irrelevant for this test




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