You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2021/02/18 16:02:02 UTC

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #871: PARQUET-1982: Random access to row groups in ParquetFileReader

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



##########
File path: parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##########
@@ -888,20 +888,44 @@ public void appendTo(ParquetFileWriter writer) throws IOException {
     writer.appendRowGroups(f, blocks, true);
   }
 
+  /**
+   * Reads all the columns requested from the row group at the specified block.
+   * @throws IOException if an error occurs while reading
+   * @return the PageReadStore which can provide PageReaders for each column.
+   */
+  public PageReadStore readRowGroup(BlockMetaData block) throws IOException {

Review comment:
       You have created the specific method to read a row group based on the specified metadata (instead simply reading the next row group). This seems fine to me. Meanwhile, there is another method for reading the next row group: `readNextFilteredRowGroup`. This method was created to support the column indexes feature and meanwhile to be backward compatible with the old readers.
   To take advantage of the column indexes feature (and skip the unnecessary pages to get read) and also for symmetry I would suggest creating another specific method: `readFilteredRowGroup(BlockMetaData)`.

##########
File path: parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+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.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.parquet.column.Encoding.BIT_PACKED;
+import static org.apache.parquet.column.Encoding.PLAIN;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private static final MessageType SCHEMA = MessageTypeParser.parseMessageType("" +
+    "message m {" +
+    "  required group a {" +
+    "    required binary b;" +
+    "  }" +
+    "  required group c {" +
+    "    required int64 d;" +
+    "  }" +
+    "}");
+  private static final String[] PATH1 = {"a", "b"};
+  private static final ColumnDescriptor C1 = SCHEMA.getColumnDescription(PATH1);
+  private static final String[] PATH2 = {"c", "d"};
+  private static final ColumnDescriptor C2 = SCHEMA.getColumnDescription(PATH2);
+
+  private static final byte[] BYTES1 = { 0, 1, 2, 3 };
+  private static final byte[] BYTES2 = { 1, 2, 3, 4 };
+  private static final byte[] BYTES3 = { 2, 3, 4, 5 };
+  private static final byte[] BYTES4 = { 3, 4, 5, 6 };
+  private static final CompressionCodecName CODEC = CompressionCodecName.UNCOMPRESSED;
+
+  private static final org.apache.parquet.column.statistics.Statistics<?> EMPTY_STATS = org.apache.parquet.column.statistics.Statistics
+    .getBuilderForReading(Types.required(PrimitiveType.PrimitiveTypeName.BINARY).named("test_binary")).build();
+
+  private void writeDataToFile(Path path) throws IOException {
+    Configuration configuration = new Configuration();
+
+    ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path);
+    w.start();
+    w.startBlock(3);
+    w.startColumn(C1, 5, CODEC);
+    w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.startColumn(C2, 6, CODEC);
+    w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.endBlock();
+    w.startBlock(4);
+    w.startColumn(C1, 7, CODEC);
+    w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.startColumn(C2, 8, CODEC);
+    w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.endBlock();
+    w.end(new HashMap<String, String>());
+  }
+
+  @Test
+  public void testReadRandom() throws IOException {
+    File testFile = temp.newFile();
+    testFile.delete();
+    Path path = new Path(testFile.toURI());
+    Configuration configuration = new Configuration();
+    ParquetReadOptions options = ParquetReadOptions.builder().build();
+
+    writeDataToFile(path);
+
+    ParquetFileReader reader = new ParquetFileReader(HadoopInputFile.fromPath(path, configuration), options);
+    List<BlockMetaData> blocks = reader.getRowGroups();
+
+    // Randomize indexes
+    List<Integer> indexes = new ArrayList<>();
+    for (int i = 0; i < blocks.size(); i++) {
+      for (int j = 0; j < 4; j++) {
+        indexes.add(i);
+      }
+    }
+
+    Collections.shuffle(indexes);

Review comment:
       This way the order of the indexes is not deterministic. Usually we don't like this in unit tests because in case of failure we cannot reproduce the test. I would suggest invoking `shuffle` with a pre-created `Random` instance with a fix seed.

##########
File path: parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderRandomAccess.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+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.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.parquet.column.Encoding.BIT_PACKED;
+import static org.apache.parquet.column.Encoding.PLAIN;
+import static org.junit.Assert.*;
+
+public class TestParquetReaderRandomAccess {
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private static final MessageType SCHEMA = MessageTypeParser.parseMessageType("" +
+    "message m {" +
+    "  required group a {" +
+    "    required binary b;" +
+    "  }" +
+    "  required group c {" +
+    "    required int64 d;" +
+    "  }" +
+    "}");
+  private static final String[] PATH1 = {"a", "b"};
+  private static final ColumnDescriptor C1 = SCHEMA.getColumnDescription(PATH1);
+  private static final String[] PATH2 = {"c", "d"};
+  private static final ColumnDescriptor C2 = SCHEMA.getColumnDescription(PATH2);
+
+  private static final byte[] BYTES1 = { 0, 1, 2, 3 };
+  private static final byte[] BYTES2 = { 1, 2, 3, 4 };
+  private static final byte[] BYTES3 = { 2, 3, 4, 5 };
+  private static final byte[] BYTES4 = { 3, 4, 5, 6 };
+  private static final CompressionCodecName CODEC = CompressionCodecName.UNCOMPRESSED;
+
+  private static final org.apache.parquet.column.statistics.Statistics<?> EMPTY_STATS = org.apache.parquet.column.statistics.Statistics
+    .getBuilderForReading(Types.required(PrimitiveType.PrimitiveTypeName.BINARY).named("test_binary")).build();
+
+  private void writeDataToFile(Path path) throws IOException {
+    Configuration configuration = new Configuration();
+
+    ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path);
+    w.start();
+    w.startBlock(3);
+    w.startColumn(C1, 5, CODEC);
+    w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.startColumn(C2, 6, CODEC);
+    w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.endBlock();
+    w.startBlock(4);
+    w.startColumn(C1, 7, CODEC);
+    w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.startColumn(C2, 8, CODEC);
+    w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN);
+    w.endColumn();
+    w.endBlock();
+    w.end(new HashMap<String, String>());
+  }
+
+  @Test
+  public void testReadRandom() throws IOException {
+    File testFile = temp.newFile();
+    testFile.delete();
+    Path path = new Path(testFile.toURI());
+    Configuration configuration = new Configuration();
+    ParquetReadOptions options = ParquetReadOptions.builder().build();
+
+    writeDataToFile(path);
+
+    ParquetFileReader reader = new ParquetFileReader(HadoopInputFile.fromPath(path, configuration), options);
+    List<BlockMetaData> blocks = reader.getRowGroups();
+
+    // Randomize indexes
+    List<Integer> indexes = new ArrayList<>();
+    for (int i = 0; i < blocks.size(); i++) {
+      for (int j = 0; j < 4; j++) {
+        indexes.add(i);
+      }
+    }
+
+    Collections.shuffle(indexes);
+
+    for (int index : indexes) {
+      PageReadStore pages = reader.readRowGroup(blocks.get(index));
+      validatePages(pages, index);
+    }
+  }
+
+  @Test
+  public void testReadRandomAndSequential() throws IOException {
+    File testFile = temp.newFile();
+    testFile.delete();
+    Path path = new Path(testFile.toURI());
+    Configuration configuration = new Configuration();
+    ParquetReadOptions options = ParquetReadOptions.builder().build();
+
+    writeDataToFile(path);
+
+    ParquetFileReader reader = new ParquetFileReader(HadoopInputFile.fromPath(path, configuration), options);
+    List<BlockMetaData> blocks = reader.getRowGroups();
+
+    // Randomize indexes
+    List<Integer> indexes = new ArrayList<>();
+    for (int i = 0; i < blocks.size(); i++) {
+      for (int j = 0; j < 4; j++) {
+        indexes.add(i);
+      }
+    }
+
+    Collections.shuffle(indexes);

Review comment:
       Same 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.

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