Skip to content

Commit

Permalink
Revert "[SPARK-35096][SQL] SchemaPruning should adhere spark.sql.case…
Browse files Browse the repository at this point in the history
…Sensitive config"

This reverts commit aef17b7.
  • Loading branch information
dongjoon-hyun committed Apr 22, 2021
1 parent b695ece commit 7af1cb4
Show file tree
Hide file tree
Showing 2 changed files with 6 additions and 52 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,9 @@

package org.apache.spark.sql.catalyst.expressions

import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.types._

object SchemaPruning extends SQLConfHelper {
object SchemaPruning {
/**
* Filters the schema by the requested fields. For example, if the schema is struct<a:int, b:int>,
* and given requested field are "a", the field "b" is pruned in the returned schema.
Expand All @@ -29,7 +28,6 @@ object SchemaPruning extends SQLConfHelper {
def pruneDataSchema(
dataSchema: StructType,
requestedRootFields: Seq[RootField]): StructType = {
val resolver = conf.resolver
// 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
Expand All @@ -38,7 +36,7 @@ object SchemaPruning extends SQLConfHelper {
.reduceLeft(_ merge _)
val dataSchemaFieldNames = dataSchema.fieldNames.toSet
val mergedDataSchema =
StructType(mergedSchema.filter(f => dataSchemaFieldNames.exists(resolver(_, f.name))))
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, dataSchema).asInstanceOf[StructType]
Expand All @@ -63,15 +61,12 @@ object SchemaPruning extends SQLConfHelper {
sortLeftFieldsByRight(leftValueType, rightValueType),
containsNull)
case (leftStruct: StructType, rightStruct: StructType) =>
val resolver = conf.resolver
val filteredRightFieldNames = rightStruct.fieldNames
.filter(name => leftStruct.fieldNames.exists(resolver(_, name)))
val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains)
val sortedLeftFields = filteredRightFieldNames.map { fieldName =>
val resolvedLeftStruct = leftStruct.find(p => resolver(p.name, fieldName)).get
val leftFieldType = resolvedLeftStruct.dataType
val leftFieldType = leftStruct(fieldName).dataType
val rightFieldType = rightStruct(fieldName).dataType
val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType)
StructField(fieldName, sortedLeftFieldType, nullable = resolvedLeftStruct.nullable)
StructField(fieldName, sortedLeftFieldType, nullable = leftStruct(fieldName).nullable)
}
StructType(sortedLeftFields)
case _ => left
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,9 @@
package org.apache.spark.sql.catalyst.expressions

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.internal.SQLConf.CASE_SENSITIVE
import org.apache.spark.sql.types._

class SchemaPruningSuite extends SparkFunSuite with SQLHelper {

def getRootFields(requestedFields: StructField*): Seq[RootField] = {
requestedFields.map { f =>
// `derivedFromAtt` doesn't affect the result of pruned schema.
SchemaPruning.RootField(field = f, derivedFromAtt = true)
}
}

class SchemaPruningSuite extends SparkFunSuite {
test("prune schema by the requested fields") {
def testPrunedSchema(
schema: StructType,
Expand Down Expand Up @@ -70,34 +59,4 @@ class SchemaPruningSuite extends SparkFunSuite with SQLHelper {
StructType.fromDDL("e int, f string")))
testPrunedSchema(complexStruct, StructField("c", IntegerType), selectFieldInMap)
}

test("SPARK-35096: test case insensitivity of pruned schema") {
Seq(true, false).foreach(isCaseSensitive => {
withSQLConf(CASE_SENSITIVE.key -> isCaseSensitive.toString) {
if (isCaseSensitive) {
// Schema is case-sensitive
val requestedFields = getRootFields(StructField("id", IntegerType))
val prunedSchema = SchemaPruning.pruneDataSchema(
StructType.fromDDL("ID int, name String"), requestedFields)
assert(prunedSchema == StructType(Seq.empty))
// Root fields are case-sensitive
val rootFieldsSchema = SchemaPruning.pruneDataSchema(
StructType.fromDDL("id int, name String"),
getRootFields(StructField("ID", IntegerType)))
assert(rootFieldsSchema == StructType(StructType(Seq.empty)))
} else {
// Schema is case-insensitive
val prunedSchema = SchemaPruning.pruneDataSchema(
StructType.fromDDL("ID int, name String"),
getRootFields(StructField("id", IntegerType)))
assert(prunedSchema == StructType(StructField("ID", IntegerType) :: Nil))
// Root fields are case-insensitive
val rootFieldsSchema = SchemaPruning.pruneDataSchema(
StructType.fromDDL("id int, name String"),
getRootFields(StructField("ID", IntegerType)))
assert(rootFieldsSchema == StructType(StructField("id", IntegerType) :: Nil))
}
}
})
}
}

0 comments on commit 7af1cb4

Please sign in to comment.