[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

shangxinli merged PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Added the file path to the log. Please take a look again. Thanks!





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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

gszadovszky commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377700950

   > @gszadovszky Nice to see you are back!
   
   @shangxinli, I wouldn't say I'm back, unfortunately. I'm a bit closer to 
Parquet at Dremio but actually not working on it. We'll see if I will have some 
spare time for it. :)




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

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


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

Review Comment:
   Same comments as above





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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -927,7 +925,15 @@ public PageReadStore readRowGroup(int blockIndex) throws 
IOException {
* @return the PageReadStore which can provide PageReaders for each column.
*/
   public PageReadStore readNextRowGroup() throws IOException {
-ColumnChunkPageReadStore rowGroup = internalReadRowGroup(currentBlock);
+ColumnChunkPageReadStore rowGroup = null;
+try {
+  rowGroup = internalReadRowGroup(currentBlock);
+} catch (ParquetEmptyBlockException e) {
+  LOG.warn("Read empty block at index {}", currentBlock);

Review Comment:
   Any way to add file path?





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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1377485663

   > Thanks you for fixing this. I've added some comments. Also, could you add 
a similar test for the filtered row groups?
   
   Thanks for your review @gszadovszky !
   
   I have addressed all of your comments. Please take a look again.




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

shangxinli commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1376751333

   @gszadovszky Nice to see you are back!




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-09 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1374806195

   @gszadovszky @ggershinsky @shangxinli @sunchao Could you please take a look 
when you have time?
   
   cc @emkornfield 




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac opened a new pull request, #1018:
URL: https://github.com/apache/parquet-mr/pull/1018

   ### Jira
   
   My PR addresses the 
[PARQUET-2219](https://issues.apache.org/jira/browse/PARQUET/PARQUET-2219).
   
   ### Tests
   
   My PR adds the following unit test to read parquet file with empty row group:
   - 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderEmptyBlock.java
   
   ### Commits
   
   The parquet specs does not forbid empty row group and some implementations 
are able to generate files with empty row group. The commit aims to make 
ParquetFileReader robust by skipping empty row group while reading.
   




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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-06 Thread Micah Kornfield (Jira)


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

Micah Kornfield commented on PARQUET-2219:
--

I'm not aware of anything in the specification that prevents zero length row 
groups.  We can try to prevent writing them out but I think readers should be 
robust to this if it isn't disallowed in the specification.   For the iterator 
case, it seems like the rowgroup should just be discarded and the next one 
checked?

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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2022-12-15 Thread Gang Wu (Jira)


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

Gang Wu commented on PARQUET-2219:
--

cc [~emkornfield]

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


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2022-12-15 Thread Gang Wu (Jira)


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

Gang Wu commented on PARQUET-2219:
--

According to the error message, it seems that empty row group is deemed as 
malformed. CMIW, some downstream readers may use the parquet reader as an 
iterator to check if it can continue reading any row. However, the empty row 
group may terminate the iterator early which is an incorrect behavior. So I 
suggest not writing the parquet file if there is no row of data.

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