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 2019/02/07 07:10:47 UTC

[GitHub] dongjoon-hyun commented on a change in pull request #23740: [SPARK-26837][SQL] Pruning nested fields from object serializers

dongjoon-hyun commented on a change in pull request #23740: [SPARK-26837][SQL] Pruning nested fields from object serializers
URL: https://github.com/apache/spark/pull/23740#discussion_r254566381
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala
 ##########
 @@ -0,0 +1,212 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+private[sql] object SchemaPruning {
+
+  /**
+   * Collects all struct types from given data type object, recursively. Supports struct and array
+   * types for now.
+   * TODO: support map type.
+   */
+  def collectStructType(dt: DataType, structs: ArrayBuffer[StructType]): ArrayBuffer[StructType] = {
+    dt match {
+      case s @ StructType(fields) =>
+        structs += s
+        fields.map(f => collectStructType(f.dataType, structs))
+      case ArrayType(elementType, _) =>
+        collectStructType(elementType, structs)
+      case _ =>
+    }
+    structs
+  }
+
+  /**
+   * This method prunes given serializer expression by given pruned data type. For example,
+   * given a serializer creating struct(a int, b int) and pruned data type struct(a int),
+   * this method returns pruned serializer creating struct(a int). For now it supports to
+   * prune nested fields in struct and array of struct.
+   * TODO: support to prune nested fields in key and value of map type.
+   */
+  def pruneSerializer(
+      serializer: NamedExpression,
+      prunedDataType: DataType): NamedExpression = {
+    val prunedStructTypes = collectStructType(prunedDataType, ArrayBuffer.empty[StructType])
+    var structTypeIndex = 0
+
+    val prunedSerializer = serializer.transformDown {
+      case s: CreateNamedStruct if structTypeIndex < prunedStructTypes.size =>
+        val prunedType = prunedStructTypes(structTypeIndex)
+
+        // Filters out the pruned fields.
+        val prunedFields = s.nameExprs.zip(s.valExprs).filter { case (nameExpr, _) =>
+          val name = nameExpr.eval(EmptyRow).toString
+          prunedType.fieldNames.exists { fieldName =>
+            if (SQLConf.get.caseSensitiveAnalysis) {
+              fieldName.equals(name)
+            } else {
+              fieldName.equalsIgnoreCase(name)
+            }
+          }
+        }.flatMap(pair => Seq(pair._1, pair._2))
+
+        structTypeIndex += 1
+        CreateNamedStruct(prunedFields)
+    }.transformUp {
+      // When we change nested serializer data type, `If` expression will be unresolved because
+      // literal null's data type doesn't match now. We need to align it with new data type.
+      // Note: we should do `transformUp` explicitly to change data types.
+      case i @ If(_: IsNull, Literal(null, dt), ser) if !dt.sameType(ser.dataType) =>
+        i.copy(trueValue = Literal(null, ser.dataType))
+    }.asInstanceOf[NamedExpression]
+
+    if (prunedSerializer.dataType.sameType(prunedDataType)) {
+      prunedSerializer
+    } else {
+      serializer
+    }
+  }
+
+  /**
+   * Filters the schema from the given file by the requested fields.
+   * Schema field ordering from the file is preserved.
+   */
+  def pruneDataSchema(
+      fileDataSchema: StructType,
+      requestedRootFields: Seq[RootField]): StructType = {
+    // Merge the requested root fields into a single schema. Note the ordering of the fields
+    // in the resulting schema may differ from their ordering in the logical relation's
+    // original schema
+    val mergedSchema = requestedRootFields
+      .map { case root: RootField => StructType(Array(root.field)) }
+      .reduceLeft(_ merge _)
+    val dataSchemaFieldNames = fileDataSchema.fieldNames.toSet
+    val mergedDataSchema =
+      StructType(mergedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
+    // Sort the fields of mergedDataSchema according to their order in dataSchema,
+    // recursively. This makes mergedDataSchema a pruned schema of dataSchema
+    sortLeftFieldsByRight(mergedDataSchema, fileDataSchema).asInstanceOf[StructType]
+  }
+
+  /**
+   * Sorts the fields and descendant fields of structs in left according to their order in
+   * right. This function assumes that the fields of left are a subset of the fields of
+   * right, recursively. That is, left is a "subschema" of right, ignoring order of
+   * fields.
+   */
+  def sortLeftFieldsByRight(left: DataType, right: DataType): DataType =
 
 Review comment:
   `def` -> `private def`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

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