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

ASF GitHub Bot commented on DRILL-5797:
---------------------------------------

Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/976#discussion_r144074677
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
 ---
    @@ -156,20 +161,46 @@ public ScanBatch getBatch(FragmentContext context, 
ParquetRowGroupScan rowGroupS
         return new ScanBatch(rowGroupScan, context, oContext, readers, 
implicitColumns);
       }
     
    -  private static boolean isComplex(ParquetMetadata footer) {
    -    MessageType schema = footer.getFileMetaData().getSchema();
    -
    -    for (Type type : schema.getFields()) {
    -      if (!type.isPrimitive()) {
    -        return true;
    +  private static boolean isComplex(ParquetMetadata footer, 
List<SchemaPath> columns) {
    +    /*
    +    ParquetRecordReader is not able to read any nested columns and is not 
able to handle repeated columns.
    +    It only handles flat column and optional column.
    +    If it is a wildcard query, we check every columns in the metadata.
    +    If not, we only check the projected columns.
    +    */
    --- End diff --
    
    Very small request: this is a great Javadoc comment, so please use this 
form:
    
    ```
    /**
     * Your comment here.
     */
    ```
    
    It may also be worth pointing out that the algorithm here works regardless 
of the form of the column:
    
    * `a`: Must consider the type of column `a` in Parquet.
    * `a.b`: The top level column `a` must be a structure in Parquet. (If not, 
then presumably an error is thrown later on.) So, no need to check `b`.
    * `a[10]`: The column `a` must be an array (repeated), so no need to check 
the column `SchemaPath` itself. Again, presumably, Drill will throw an error 
internally if it turns out that `a` is not an array.


> Use more often the new parquet reader
> -------------------------------------
>
>                 Key: DRILL-5797
>                 URL: https://issues.apache.org/jira/browse/DRILL-5797
>             Project: Apache Drill
>          Issue Type: Improvement
>          Components: Storage - Parquet
>            Reporter: Damien Profeta
>            Assignee: Damien Profeta
>              Labels: ready-to-commit
>             Fix For: 1.12.0
>
>
> The choice of using the regular parquet reader of the optimized one is based 
> of what type of columns is in the file. But the columns that are read by the 
> query doesn't matter. We can increase a little bit the cases where the 
> optimized reader is used by checking is the projected column are simple or 
> not.
> This is an optimization waiting for the fast parquet reader to handle complex 
> structure.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to