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 2022/06/09 04:00:56 UTC

[GitHub] [parquet-mr] LuciferYang commented on a diff in pull request #972: PARQUET-2154: `ParquetFileReader` should close its input stream when `filterRowGroups` throw Exception in constructor

LuciferYang commented on code in PR #972:
URL: https://github.com/apache/parquet-mr/pull/972#discussion_r893047755


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##########
@@ -715,7 +715,14 @@ public ParquetFileReader(
                                       .withDecryption(fileDecryptor.getDecryptionProperties())
                                       .build();
     }
-    this.blocks = filterRowGroups(blocks);
+    try {
+      this.blocks = filterRowGroups(blocks);
+    } catch (Exception e) {

Review Comment:
   There will be more than IOE here, for example if push down filters for repeated primitive types will throw `IllegalArgumentException` as follows:
   
   ```
   Caused by: java.lang.IllegalArgumentException: FilterPredicates do not currently support repeated columns. Column keywords is repeated.
     at org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator.validateColumn(SchemaCompatibilityValidator.java:176)
     at org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator.validateColumnFilterPredicate(SchemaCompatibilityValidator.java:149)
     at org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator.visit(SchemaCompatibilityValidator.java:89)
     at org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator.visit(SchemaCompatibilityValidator.java:56)
     at org.apache.parquet.filter2.predicate.Operators$NotEq.accept(Operators.java:192)
     at org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator.validate(SchemaCompatibilityValidator.java:61)
     at org.apache.parquet.filter2.compat.RowGroupFilter.visit(RowGroupFilter.java:95)
     at org.apache.parquet.filter2.compat.RowGroupFilter.visit(RowGroupFilter.java:45)
     at org.apache.parquet.filter2.compat.FilterCompat$FilterPredicateCompat.accept(FilterCompat.java:149)
     at org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups(RowGroupFilter.java:72)
     at org.apache.parquet.hadoop.ParquetFileReader.filterRowGroups(ParquetFileReader.java:870)
     at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:789)
   ``` 
   
   



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

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

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