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

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

gszadovszky commented on code in PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#discussion_r1064374553


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##########
@@ -1038,7 +1044,9 @@ public PageReadStore readNextFilteredRowGroup() throws 
IOException {
     }
     BlockMetaData block = blocks.get(currentBlock);
     if (block.getRowCount() == 0L) {
-      throw new RuntimeException("Illegal row group of 0 rows");
+      // Skip the empty block
+      advanceToNextBlock();
+      return readNextFilteredRowGroup();

Review Comment:
   There is a warning log in case of `readNextRowGroup` but here we don't log 
anything.



##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderEmptyBlock.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+public class TestParquetReaderEmptyBlock {
+
+  private static final Path EMPTY_BLOCK_FILE_1 = 
createPathFromCP("/test-empty-row-group_1.parquet");
+
+  private static final Path EMPTY_BLOCK_FILE_2 = 
createPathFromCP("/test-empty-row-group_2.parquet");
+
+  private static Path createPathFromCP(String path) {
+    try {
+      return new 
Path(TestParquetReaderEmptyBlock.class.getResource(path).toURI());
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testReadOnlyEmptyBlock() throws IOException {
+    Configuration conf = new Configuration();
+    ParquetMetadata readFooter = ParquetFileReader.readFooter(conf, 
EMPTY_BLOCK_FILE_1);
+
+    // The parquet file contains only one empty row group
+    Assert.assertEquals(1, readFooter.getBlocks().size());
+
+    // The empty block is skipped
+    try (ParquetFileReader r = new ParquetFileReader(conf, EMPTY_BLOCK_FILE_1, 
readFooter)) {
+      Assert.assertNull(r.readNextRowGroup());
+    }
+  }
+
+  @Test
+  public void testSkipEmptyBlock() throws IOException {
+    Configuration conf = new Configuration();
+    ParquetMetadata readFooter = ParquetFileReader.readFooter(conf, 
EMPTY_BLOCK_FILE_2);
+
+    // The parquet file contains three row groups, the second one is empty

Review Comment:
   I think, it would be nice to test the case of multiple empty row groups next 
to each other.





> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> -----------------------------------------------------------------------------------------------
>
>                 Key: PARQUET-2219
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2219
>             Project: Parquet
>          Issue Type: Bug
>          Components: parquet-mr
>    Affects Versions: 1.12.1
>            Reporter: chris stockton
>            Assignee: Gang Wu
>            Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to