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

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

arina-ielchiieva commented on a change in pull request #1370: DRILL-5797: Use 
Parquet new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201677000
 
 

 ##########
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
 ##########
 @@ -525,4 +599,57 @@ public static long getDateTimeValueFromBinary(Binary 
binaryTimeStampValue, boole
     }
   }
 
+  /**
+   * Check whether any of columns in the given list is either nested or 
repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List<SchemaPath> columns) {
+
+    MessageType schema = footer.getFileMetaData().getSchema();
+
+    if (Utilities.isStarQuery(columns)) {
+      for (Type type : schema.getFields()) {
+        if (!type.isPrimitive()) {
+          return true;
+        }
+      }
+      for (ColumnDescriptor col : schema.getColumns()) {
+        if (col.getMaxRepetitionLevel() > 0) {
+          return true;
+        }
+      }
+      return false;
+    } else {
+      Map<String, ColumnDescriptor> colDescMap = 
ParquetReaderUtility.getColNameToColumnDescriptorMapping(footer);
+      Map<String, SchemaElement> schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+
+      for (SchemaPath schemaPath : columns) {
+        // Schema path which is non-leaf is complex column
+        if (!schemaPath.isLeaf()) {
+          logger.debug("rowGroupScan contains complex column: {}", 
schemaPath.getUnIndexed().toString());
+          return true;
+        }
+
+        // following column descriptor lookup failure may mean two cases, 
depending on subsequent SchemaElement lookup:
+        // 1. success: queried column is complex, i.e. GroupType
+        // 2. failure: queried column is not in schema and thus is non-complex
+        ColumnDescriptor column = 
colDescMap.get(schemaPath.getUnIndexed().toString().toLowerCase());
+
+        if (column == null) {
+          SchemaElement schemaElement = 
schemaElements.get(schemaPath.getUnIndexed().toString().toLowerCase());
+          if (schemaElement != null) {
+            return true;
+          }
+        } else {
+          if (column.getMaxRepetitionLevel() > 0) {
+            logger.debug("rowGroupScan contains repetitive column: {}", 
schemaPath.getUnIndexed().toString());
 
 Review comment:
   trace?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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: Oleksandr Kalinin
>            Priority: Major
>             Fix For: 1.15.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
(v7.6.3#76005)

Reply via email to