You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/09 16:12:38 UTC

[GitHub] [spark] maropu commented on a change in pull request #28761: [SPARK-25557][SQL] Nested column predicate pushdown for ORC

maropu commented on a change in pull request #28761:
URL: https://github.com/apache/spark/pull/28761#discussion_r437279326



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala
##########
@@ -37,12 +40,44 @@ trait OrcFiltersBase {
   }
 
   /**
-   * Return true if this is a searchable type in ORC.
-   * Both CharType and VarcharType are cleaned at AstBuilder.
+   * This method returns a map which contains ORC field name and data type. Each key
+   * represents a column; `dots` are used as separators for nested columns. If any part
+   * of the names contains `dots`, it is quoted to avoid confusion. See
+   * `org.apache.spark.sql.connector.catalog.quote` for implementation details.
    */
-  protected[sql] def isSearchableType(dataType: DataType) = dataType match {
-    case BinaryType => false
-    case _: AtomicType => true
-    case _ => false
+  protected[sql] def getNameToOrcFieldMap(
+      schema: StructType,
+      caseSensitive: Boolean): Map[String, DataType] = {
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+
+    def getPrimitiveFields(
+        fields: Seq[StructField],
+        parentFieldNames: Array[String] = Array.empty): Seq[(String, DataType)] = {
+      fields.flatMap { f =>
+        f.dataType match {
+          case st: StructType =>
+            getPrimitiveFields(st.fields.toSeq, parentFieldNames :+ f.name)
+          case BinaryType => None
+          case _: AtomicType =>
+            Some(((parentFieldNames :+ f.name).toSeq.quoted, f.dataType))

Review comment:
       We don't need `toSeq` if we use `Seq` for `parentFieldNames`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2080,9 +2080,9 @@ object SQLConf {
       .doc("A comma-separated list of data source short names or fully qualified data source " +
         "implementation class names for which Spark tries to push down predicates for nested " +
         "columns and/or names containing `dots` to data sources. This configuration is only " +
-        "effective with file-based data source in DSv1. Currently, Parquet implements " +
-        "both optimizations while ORC only supports predicates for names containing `dots`. The " +
-        "other data sources don't support this feature yet. So the default value is 'parquet,orc'.")
+        "effective with file-based data source in DSv1. Currently, Parquet and ORC implement " +

Review comment:
       nit: (this is not related to this PR though..) `data source` -> `data sources`?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala
##########
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.orc
 import scala.collection.JavaConverters._
 
 import org.apache.spark.SparkConf
-import org.apache.spark.sql.{Column, DataFrame}
+import org.apache.spark.sql.{Column, DataFrame, Row}

Review comment:
       revert this back?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala
##########
@@ -37,12 +40,44 @@ trait OrcFiltersBase {
   }
 
   /**
-   * Return true if this is a searchable type in ORC.
-   * Both CharType and VarcharType are cleaned at AstBuilder.
+   * This method returns a map which contains ORC field name and data type. Each key
+   * represents a column; `dots` are used as separators for nested columns. If any part
+   * of the names contains `dots`, it is quoted to avoid confusion. See
+   * `org.apache.spark.sql.connector.catalog.quote` for implementation details.
    */
-  protected[sql] def isSearchableType(dataType: DataType) = dataType match {
-    case BinaryType => false
-    case _: AtomicType => true
-    case _ => false
+  protected[sql] def getNameToOrcFieldMap(
+      schema: StructType,
+      caseSensitive: Boolean): Map[String, DataType] = {
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper

Review comment:
       Any reason not to put this import in the top?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala
##########
@@ -37,12 +40,44 @@ trait OrcFiltersBase {
   }
 
   /**
-   * Return true if this is a searchable type in ORC.
-   * Both CharType and VarcharType are cleaned at AstBuilder.
+   * This method returns a map which contains ORC field name and data type. Each key
+   * represents a column; `dots` are used as separators for nested columns. If any part
+   * of the names contains `dots`, it is quoted to avoid confusion. See
+   * `org.apache.spark.sql.connector.catalog.quote` for implementation details.
    */
-  protected[sql] def isSearchableType(dataType: DataType) = dataType match {
-    case BinaryType => false
-    case _: AtomicType => true
-    case _ => false
+  protected[sql] def getNameToOrcFieldMap(
+      schema: StructType,
+      caseSensitive: Boolean): Map[String, DataType] = {
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+
+    def getPrimitiveFields(
+        fields: Seq[StructField],
+        parentFieldNames: Array[String] = Array.empty): Seq[(String, DataType)] = {

Review comment:
       `Array` -> `Seq`?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org