From 6e41081e98be40fde7f9099670c4ef3a84531cf0 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Fri, 24 Jun 2016 10:21:24 -0700 Subject: [PATCH 01/24] [SPARK-4502][SQL] Parquet nested column pruning --- .../planning/GetStructFieldObject.scala | 33 ++ .../planning/ProjectionOverSchema.scala | 62 +++ .../sql/catalyst/planning/SelectedField.scala | 134 ++++++ .../plans/logical/QueryPlanConstraints.scala | 22 +- .../apache/spark/sql/internal/SQLConf.scala | 14 +- .../sql/catalyst/SchemaPruningTest.scala | 52 +++ .../planning/SelectedFieldSuite.scala | 432 ++++++++++++++++++ .../sql/execution/DataSourceScanExec.scala | 14 +- .../spark/sql/execution/SparkOptimizer.scala | 4 +- .../execution/basicPhysicalOperators.scala | 4 +- .../datasources/ColumnarFileFormat.scala | 32 ++ .../parquet/ParquetFileFormat.scala | 14 +- .../parquet/ParquetReadSupport.scala | 57 ++- .../parquet/ParquetRowConverter.scala | 22 +- .../parquet/ParquetSchemaPruning.scala | 154 +++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 15 + .../sql/execution/FileSchemaPruningTest.scala | 54 +++ .../parquet/ParquetQuerySuite.scala | 15 +- .../parquet/ParquetSchemaPruningSuite.scala | 124 +++++ .../datasources/parquet/ParquetTest.scala | 48 +- 20 files changed, 1261 insertions(+), 45 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala new file mode 100644 index 0000000000000..033792a9ac728 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala @@ -0,0 +1,33 @@ +/* + * 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.planning + +import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField} +import org.apache.spark.sql.types.StructField + +/** + * A Scala extractor that extracts the child expression and struct field from a [[GetStructField]]. + * This is in contrast to the [[GetStructField]] case class extractor which returns the field + * ordinal instead of the field itself. + */ +private[planning] object GetStructFieldObject { + def unapply(getStructField: GetStructField): Option[(Expression, StructField)] = + Some(( + getStructField.child, + getStructField.childSchema(getStructField.ordinal))) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala new file mode 100644 index 0000000000000..e305676ffa8a1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala @@ -0,0 +1,62 @@ +/* + * 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.planning + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +/** + * A Scala extractor that projects an expression over a given schema. Data types, + * field indexes and field counts of complex type extractors and attributes + * are adjusted to fit the schema. All other expressions are left as-is. This + * class is motivated by columnar nested schema pruning. + */ +case class ProjectionOverSchema(schema: StructType) { + private val fieldNames = schema.fieldNames.toSet + + def unapply(expr: Expression): Option[Expression] = getProjection(expr) + + private def getProjection(expr: Expression): Option[Expression] = + expr match { + case a @ AttributeReference(name, _, _, _) if (fieldNames.contains(name)) => + Some(a.copy(dataType = schema(name).dataType)(a.exprId, a.qualifier)) + case GetArrayItem(child, arrayItemOrdinal) => + getProjection(child).map { + case projection => + GetArrayItem(projection, arrayItemOrdinal) + } + case GetArrayStructFields(child, StructField(name, _, _, _), _, numFields, containsNull) => + getProjection(child).map(p => (p, p.dataType)).map { + case (projection, ArrayType(projSchema @ StructType(_), _)) => + GetArrayStructFields(projection, + projSchema(name), projSchema.fieldIndex(name), projSchema.size, containsNull) + } + case GetMapValue(child, key) => + getProjection(child).map { + case projection => + GetMapValue(projection, key) + } + case GetStructFieldObject(child, StructField(name, _, _, _)) => + getProjection(child).map(p => (p, p.dataType)).map { + case (projection, projSchema @ StructType(_)) => + GetStructField(projection, projSchema.fieldIndex(name)) + } + case _ => + None + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala new file mode 100644 index 0000000000000..dc1e00290bedb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala @@ -0,0 +1,134 @@ +/* + * 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.planning + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +/** + * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst + * complex type extractor. For example, consider a relation with the following schema: + * + * {{{ + * root + * |-- name: struct (nullable = true) + * | |-- first: string (nullable = true) + * | |-- last: string (nullable = true) + * }}} + * + * Further, suppose we take the select expression `name.first`. This will parse into an + * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern: + * + * {{{ + * GetStructFieldObject( + * AttributeReference("name", StructType(_), _, _), + * StructField("first", StringType, _, _)) + * }}} + * + * [[SelectedField]] converts that expression into + * + * {{{ + * StructField("name", StructType(Array(StructField("first", StringType)))) + * }}} + * + * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the + * same name as its child (or "parent" going right to left in the select expression) and a data + * type appropriate to the complex type extractor. In our example, the name of the child expression + * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string + * field named "first". + * + * @param expr the top-level complex type extractor + */ +object SelectedField { + def unapply(expr: Expression): Option[StructField] = { + // If this expression is an alias, work on its child instead + val unaliased = expr match { + case Alias(child, _) => child + case expr => expr + } + selectField(unaliased, None) + } + + private def selectField(expr: Expression, fieldOpt: Option[StructField]): Option[StructField] = { + expr match { + // No children. Returns a StructField with the attribute name or None if fieldOpt is None. + case AttributeReference(name, dataType, nullable, metadata) => + fieldOpt.map(field => + StructField(name, wrapStructType(dataType, field), nullable, metadata)) + // Handles case "expr0.field[n]", where "expr0" is of struct type and "expr0.field" is of + // array type. + case GetArrayItem(x @ GetStructFieldObject(child, field @ StructField(name, + dataType, nullable, metadata)), _) => + val childField = fieldOpt.map(field => StructField(name, + wrapStructType(dataType, field), nullable, metadata)).getOrElse(field) + selectField(child, Some(childField)) + // Handles case "expr0.field[n]", where "expr0.field" is of array type. + case GetArrayItem(child, _) => + selectField(child, fieldOpt) + // Handles case "expr0.field.subfield", where "expr0" and "expr0.field" are of array type. + case GetArrayStructFields(child: GetArrayStructFields, + field @ StructField(name, dataType, nullable, metadata), _, _, _) => + val childField = fieldOpt.map(field => StructField(name, + wrapStructType(dataType, field), + nullable, metadata)).getOrElse(field) + selectField(child, Some(childField)) + // Handles case "expr0.field", where "expr0" is of array type. + case GetArrayStructFields(child, + field @ StructField(name, dataType, nullable, metadata), _, _, containsNull) => + val childField = + fieldOpt.map(field => StructField(name, + wrapStructType(dataType, field), + nullable, metadata)).getOrElse(field) + selectField(child, Some(childField)) + // Handles case "expr0.field[key]", where "expr0" is of struct type and "expr0.field" is of + // map type. + case GetMapValue(x @ GetStructFieldObject(child, field @ StructField(name, + dataType, + nullable, metadata)), _) => + val childField = fieldOpt.map(field => StructField(name, + wrapStructType(dataType, field), + nullable, metadata)).getOrElse(field) + selectField(child, Some(childField)) + // Handles case "expr0.field[key]", where "expr0.field" is of map type. + case GetMapValue(child, _) => + selectField(child, fieldOpt) + // Handles case "expr0.field", where expr0 is of struct type. + case GetStructFieldObject(child, + field @ StructField(name, dataType, nullable, metadata)) => + val childField = fieldOpt.map(field => StructField(name, + wrapStructType(dataType, field), + nullable, metadata)).getOrElse(field) + selectField(child, Some(childField)) + case _ => + None + } + } + + // Constructs a composition of complex types with a StructType(Array(field)) at its core. Returns + // a StructType for a StructType, an ArrayType for an ArrayType and a MapType for a MapType. + private def wrapStructType(dataType: DataType, field: StructField): DataType = { + dataType match { + case _: StructType => + StructType(Array(field)) + case ArrayType(elementType, containsNull) => + ArrayType(wrapStructType(elementType, field), containsNull) + case MapType(keyType, valueType, valueContainsNull) => + MapType(keyType, wrapStructType(valueType, field), valueContainsNull) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index cc352c59dff80..5bfb2ec4c678b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -81,8 +81,9 @@ trait ConstraintHelper { /** * Infers a set of `isNotNull` constraints from null intolerant expressions as well as - * non-nullable attributes. For e.g., if an expression is of the form (`a > 5`), this - * returns a constraint of the form `isNotNull(a)` + * non-nullable attributes and complex type extractors. For example, if an expression is of the + * form (`a > 5`), this returns a constraint of the form `isNotNull(a)`. For an expression of the + * form (`a.b > 5`), this returns the more precise constraint `isNotNull(a.b)`. */ def constructIsNotNullConstraints( constraints: Set[Expression], @@ -99,27 +100,28 @@ trait ConstraintHelper { } /** - * Infer the Attribute-specific IsNotNull constraints from the null intolerant child expressions - * of constraints. + * Infer the Attribute and ExtractValue-specific IsNotNull constraints from the null intolerant + * child expressions of constraints. */ private def inferIsNotNullConstraints(constraint: Expression): Seq[Expression] = constraint match { // When the root is IsNotNull, we can push IsNotNull through the child null intolerant // expressions - case IsNotNull(expr) => scanNullIntolerantAttribute(expr).map(IsNotNull(_)) + case IsNotNull(expr) => scanNullIntolerantField(expr).map(IsNotNull(_)) // Constraints always return true for all the inputs. That means, null will never be returned. // Thus, we can infer `IsNotNull(constraint)`, and also push IsNotNull through the child // null intolerant expressions. - case _ => scanNullIntolerantAttribute(constraint).map(IsNotNull(_)) + case _ => scanNullIntolerantField(constraint).map(IsNotNull(_)) } /** - * Recursively explores the expressions which are null intolerant and returns all attributes - * in these expressions. + * Recursively explores the expressions which are null intolerant and returns all attributes and + * complex type extractors in these expressions. */ - private def scanNullIntolerantAttribute(expr: Expression): Seq[Attribute] = expr match { + private def scanNullIntolerantField(expr: Expression): Seq[Expression] = expr match { + case ev: ExtractValue => Seq(ev) case a: Attribute => Seq(a) - case _: NullIntolerant => expr.children.flatMap(scanNullIntolerantAttribute) + case _: NullIntolerant => expr.children.flatMap(scanNullIntolerantField) case _ => Seq.empty[Attribute] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 979a55467ff89..6a3b7df5e0be5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1381,8 +1381,18 @@ object SQLConf { "issues. Turn on this config to insert a local sort before actually doing repartition " + "to generate consistent repartition results. The performance of repartition() may go " + "down since we insert extra local sort before it.") + .booleanConf + .createWithDefault(true) + + val NESTED_SCHEMA_PRUNING_ENABLED = + buildConf("spark.sql.nestedSchemaPruning.enabled") + .internal() + .doc("Prune nested fields from a logical relation's output which are unnecessary in " + + "satisfying a query. This optimization allows columnar file format readers to avoid " + + "reading unnecessary nested column data. Currently Parquet is the only data source that " + + "implements this optimization.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") @@ -1863,6 +1873,8 @@ class SQLConf extends Serializable with Logging { def partitionOverwriteMode: PartitionOverwriteMode.Value = PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE)) + def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) + def csvColumnPruning: Boolean = getConf(SQLConf.CSV_PARSER_COLUMN_PRUNING) def legacySizeOfNull: Boolean = getConf(SQLConf.LEGACY_SIZE_OF_NULL) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala new file mode 100644 index 0000000000000..88e30e03d2d3b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED + +/** + * A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled. + * Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true. + */ +private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll { + private var originalConfSchemaPruningEnabled = false + + override protected def beforeAll(): Unit = { + // Call `withSQLConf` eagerly because some subtypes of `PlanTest` (I'm looking at you, + // `SQLTestUtils`) override `withSQLConf` to reset the existing `SQLConf` with a new one without + // copying existing settings first. This here is an awful, ugly way to get around that behavior + // by initializing the "real" `SQLConf` with an noop call to `withSQLConf`. I don't want to risk + // "fixing" the downstream behavior, breaking everything else that's expecting these semantics. + // Oh well... + withSQLConf()(()) + originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true) + super.beforeAll() + } + + override protected def afterAll(): Unit = { + try { + super.afterAll() + } finally { + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala new file mode 100644 index 0000000000000..6923c7a89abf9 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala @@ -0,0 +1,432 @@ +/* + * 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.planning + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.types._ + +// scalastyle:off line.size.limit +class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { + // The test schema as a tree string, i.e. `schema.treeString` + // root + // |-- col1: string (nullable = false) + // |-- col2: struct (nullable = true) + // | |-- field1: integer (nullable = true) + // | |-- field2: array (nullable = true) + // | | |-- element: integer (containsNull = false) + // | |-- field3: array (nullable = false) + // | | |-- element: struct (containsNull = true) + // | | | |-- subfield1: integer (nullable = true) + // | | | |-- subfield2: integer (nullable = true) + // | | | |-- subfield3: array (nullable = true) + // | | | | |-- element: integer (containsNull = true) + // | |-- field4: map (nullable = true) + // | | |-- key: string + // | | |-- value: struct (valueContainsNull = false) + // | | | |-- subfield1: integer (nullable = true) + // | | | |-- subfield2: array (nullable = true) + // | | | | |-- element: integer (containsNull = false) + // | |-- field5: array (nullable = false) + // | | |-- element: struct (containsNull = true) + // | | | |-- subfield1: struct (nullable = false) + // | | | | |-- subsubfield1: integer (nullable = true) + // | | | | |-- subsubfield2: integer (nullable = true) + // | | | |-- subfield2: struct (nullable = true) + // | | | | |-- subsubfield1: struct (nullable = true) + // | | | | | |-- subsubsubfield1: string (nullable = true) + // | | | | |-- subsubfield2: integer (nullable = true) + // | |-- field6: struct (nullable = true) + // | | |-- subfield1: string (nullable = false) + // | | |-- subfield2: string (nullable = true) + // | |-- field7: struct (nullable = true) + // | | |-- subfield1: struct (nullable = true) + // | | | |-- subsubfield1: integer (nullable = true) + // | | | |-- subsubfield2: integer (nullable = true) + // | |-- field8: map (nullable = true) + // | | |-- key: string + // | | |-- value: array (valueContainsNull = false) + // | | | |-- element: struct (containsNull = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: array (nullable = true) + // | | | | | |-- element: integer (containsNull = false) + // | |-- field9: map (nullable = true) + // | | |-- key: string + // | | |-- value: integer (valueContainsNull = false) + // |-- col3: array (nullable = false) + // | |-- element: struct (containsNull = false) + // | | |-- field1: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | | |-- subfield2: integer (nullable = true) + // | | |-- field2: map (nullable = true) + // | | | |-- key: string + // | | | |-- value: integer (valueContainsNull = false) + // |-- col4: map (nullable = false) + // | |-- key: string + // | |-- value: struct (valueContainsNull = false) + // | | |-- field1: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | | |-- subfield2: integer (nullable = true) + // | | |-- field2: map (nullable = true) + // | | | |-- key: string + // | | | |-- value: integer (valueContainsNull = false) + // |-- col5: array (nullable = true) + // | |-- element: map (containsNull = true) + // | | |-- key: string + // | | |-- value: struct (valueContainsNull = false) + // | | | |-- field1: struct (nullable = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: integer (nullable = true) + // |-- col6: map (nullable = true) + // | |-- key: string + // | |-- value: array (valueContainsNull = true) + // | | |-- element: struct (containsNull = false) + // | | | |-- field1: struct (nullable = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: integer (nullable = true) + // |-- col7: array (nullable = true) + // | |-- element: struct (containsNull = true) + // | | |-- field1: integer (nullable = false) + // | | |-- field2: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | |-- field3: array (nullable = true) + // | | | |-- element: struct (containsNull = true) + // | | | | |-- subfield1: integer (nullable = false) + // |-- col8: array (nullable = true) + // | |-- element: struct (containsNull = true) + // | | |-- field1: array (nullable = false) + // | | | |-- element: integer (containsNull = false) + private val schema = + StructType( + StructField("col1", StringType, nullable = false) :: + StructField("col2", StructType( + StructField("field1", IntegerType) :: + StructField("field2", ArrayType(IntegerType, containsNull = false)) :: + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: + StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: + StructField("field4", MapType(StringType, StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: + StructField("subfield2", StructType( + StructField("subsubfield1", StructType( + StructField("subsubsubfield1", StringType) :: Nil)) :: + StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) :: + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: + StructField("subfield2", StringType) :: Nil)) :: + StructField("field7", StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: + StructField("field8", MapType(StringType, ArrayType(StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) :: + StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) :: + StructField("col3", ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: + StructField("subfield2", IntegerType) :: Nil)) :: + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) :: + StructField("col4", MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: + StructField("subfield2", IntegerType) :: Nil)) :: + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) :: + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) :: + StructField("col6", MapType(StringType, ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) :: + StructField("col7", ArrayType(StructType( + StructField("field1", IntegerType, nullable = false) :: + StructField("field2", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) :: + StructField("col8", ArrayType(StructType( + StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil) + + private val testRelation = LocalRelation(schema.toAttributes) + + test("should not match an attribute reference") { + assertResult(None)(unapplySelect("col1")) + assertResult(None)(unapplySelect("col1 as foo")) + assertResult(None)(unapplySelect("col2")) + } + + test("col2.field2, col2.field2[0] as foo") { + val expected = + StructField("col2", StructType( + StructField("field2", ArrayType(IntegerType, containsNull = false)) :: Nil)) + testSelect("col2.field2", expected) + testSelect("col2.field2[0] as foo", expected) + } + + test("col2.field9, col2.field9['foo'] as foo") { + val expected = + StructField("col2", StructType( + StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) + testSelect("col2.field9", expected) + testSelect("col2.field9['foo'] as foo", expected) + } + + test("col2.field3.subfield3, col2.field3[0].subfield3 as foo, col2.field3.subfield3[0] as foo, col2.field3[0].subfield3[0] as foo") { + val expected = + StructField("col2", StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: Nil)) + testSelect("col2.field3.subfield3", expected) + testSelect("col2.field3[0].subfield3 as foo", expected) + testSelect("col2.field3.subfield3[0] as foo", expected) + testSelect("col2.field3[0].subfield3[0] as foo", expected) + } + + test("col2.field3.subfield1") { + val expected = + StructField("col2", StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType) :: Nil)), nullable = false) :: Nil)) + testSelect("col2.field3.subfield1", expected) + } + + test("col2.field5.subfield1") { + val expected = + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) + testSelect("col2.field5.subfield1", expected) + } + + test("col3.field1.subfield1") { + val expected = + StructField("col3", ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), containsNull = false), nullable = false) + testSelect("col3.field1.subfield1", expected) + } + + test("col3.field2['foo'] as foo") { + val expected = + StructField("col3", ArrayType(StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) + testSelect("col3.field2['foo'] as foo", expected) + } + + test("col4['foo'].field1.subfield1 as foo") { + val expected = + StructField("col4", MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), valueContainsNull = false), nullable = false) + testSelect("col4['foo'].field1.subfield1 as foo", expected) + } + + test("col4['foo'].field2['bar'] as foo") { + val expected = + StructField("col4", MapType(StringType, StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) + testSelect("col4['foo'].field2['bar'] as foo", expected) + } + + test("col5[0]['foo'].field1.subfield1 as foo") { + val expected = + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + testSelect("col5[0]['foo'].field1.subfield1 as foo", expected) + } + + test("col6['foo'][0].field1.subfield1 as foo") { + val expected = + StructField("col6", MapType(StringType, ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false))) + testSelect("col6['foo'][0].field1.subfield1 as foo", expected) + } + + test("col2.field5.subfield1.subsubfield1") { + val expected = + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) + testSelect("col2.field5.subfield1.subsubfield1", expected) + } + + test("col2.field5.subfield2.subsubfield1.subsubsubfield1") { + val expected = + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield2", StructType( + StructField("subsubfield1", StructType( + StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) :: Nil)), nullable = false) :: Nil)) + testSelect("col2.field5.subfield2.subsubfield1.subsubsubfield1", expected) + } + + test("col2.field4['foo'].subfield1 as foo") { + val expected = + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil)) + testSelect("col2.field4['foo'].subfield1 as foo", expected) + } + + test("col2.field4['foo'].subfield2 as foo, col2.field4['foo'].subfield2[0] as foo") { + val expected = + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: Nil)) + testSelect("col2.field4['foo'].subfield2 as foo", expected) + testSelect("col2.field4['foo'].subfield2[0] as foo", expected) + } + + test("col2.field8['foo'][0].subfield1 as foo") { + val expected = + StructField("col2", StructType( + StructField("field8", MapType(StringType, ArrayType(StructType( + StructField("subfield1", IntegerType) :: Nil)), valueContainsNull = false)) :: Nil)) + testSelect("col2.field8['foo'][0].subfield1 as foo", expected) + } + + test("col2.field1") { + val expected = + StructField("col2", StructType( + StructField("field1", IntegerType) :: Nil)) + testSelect("col2.field1", expected) + } + + test("col2.field6") { + val expected = + StructField("col2", StructType( + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: + StructField("subfield2", StringType) :: Nil)) :: Nil)) + testSelect("col2.field6", expected) + } + + test("col2.field7.subfield1") { + val expected = + StructField("col2", StructType( + StructField("field7", StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: Nil)) + testSelect("col2.field7.subfield1", expected) + } + + test("col2.field6.subfield1") { + val expected = + StructField("col2", StructType( + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil)) + testSelect("col2.field6.subfield1", expected) + } + + test("col7.field1, col7[0].field1 as foo, col7.field1[0] as foo") { + val expected = + StructField("col7", ArrayType(StructType( + StructField("field1", IntegerType, nullable = false) :: Nil))) + testSelect("col7.field1", expected) + testSelect("col7[0].field1 as foo", expected) + testSelect("col7.field1[0] as foo", expected) + } + + test("col7.field2.subfield1") { + val expected = + StructField("col7", ArrayType(StructType( + StructField("field2", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil))) + testSelect("col7.field2.subfield1", expected) + } + + test("col7.field3.subfield1") { + val expected = + StructField("col7", ArrayType(StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) + testSelect("col7.field3.subfield1", expected) + } + + test("col8.field1, col8[0].field1 as foo, col8.field1[0] as foo, col8[0].field1[0] as foo") { + val expected = + StructField("col8", ArrayType(StructType( + StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) + testSelect("col8.field1", expected) + testSelect("col8[0].field1 as foo", expected) + testSelect("col8.field1[0] as foo", expected) + testSelect("col8[0].field1[0] as foo", expected) + } + + def assertResult(expected: StructField)(actual: StructField)(expr: String): Unit = { + try { + super.assertResult(expected)(actual) + } catch { + case ex: TestFailedException => + // Print some helpful diagnostics in the case of failure + // scalastyle:off println + println("For " + expr) + println("Expected:") + println(StructType(expected :: Nil).treeString) + println("Actual:") + println(StructType(actual :: Nil).treeString) + println("expected.dataType.sameType(actual.dataType) = " + + expected.dataType.sameType(actual.dataType)) + // scalastyle:on println + throw ex + } + } + + private def testSelect(expr: String, expected: StructField) = { + unapplySelect(expr) match { + case Some(field) => + assertResult(expected)(field)(expr) + case None => + val failureMessage = + "Failed to select a field from " + expr + ". " + + "Expected:\n" + + StructType(expected :: Nil).treeString + fail(failureMessage) + } + } + + private def unapplySelect(expr: String) = { + val parsedExpr = + CatalystSqlParser.parseExpression(expr) match { + case namedExpr: NamedExpression => namedExpr + } + val select = testRelation.select(parsedExpr) + val analyzed = select.analyze + SelectedField.unapply(analyzed.expressions.head) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 36ed016773b67..0bf7248b06e9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -304,7 +304,19 @@ case class FileSourceScanExec( withOptPartitionCount } - withSelectedBucketsCount + val withOptColumnCount = relation.fileFormat match { + case columnar: ColumnarFileFormat => + SparkSession + .getActiveSession + .map { sparkSession => + val columnCount = columnar.columnCountForSchema(sparkSession, requiredSchema) + withSelectedBucketsCount + ("ColumnCount" -> columnCount.toString) + } getOrElse { + withSelectedBucketsCount + } + case _ => withSelectedBucketsCount + } + withOptColumnCount } private lazy val inputRDD: RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 64d3f2cdbfa82..969def7624058 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaPruning import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate class SparkOptimizer( @@ -31,7 +32,8 @@ class SparkOptimizer( override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ - Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions)) ++ + Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ + Batch("Parquet Schema Pruning", Once, ParquetSchemaPruning)) ++ postHocOptimizationBatches :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 9434ceb7cd16c..c29c336a974f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -162,7 +162,9 @@ case class FilterExec(condition: Expression, child: SparkPlan) val generatedIsNotNullChecks = new Array[Boolean](notNullPreds.length) val generated = otherPreds.map { c => val nullChecks = c.references.map { r => - val idx = notNullPreds.indexWhere { n => n.asInstanceOf[IsNotNull].child.semanticEquals(r)} + val idx = notNullPreds.indexWhere { n => + n.asInstanceOf[IsNotNull].child.references.contains(r) + } if (idx != -1 && !generatedIsNotNullChecks(idx)) { generatedIsNotNullChecks(idx) = true // Use the child's output. The nullability is what the child produced. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala new file mode 100644 index 0000000000000..ee0726cb2f00a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala @@ -0,0 +1,32 @@ +/* + * 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.execution.datasources + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + +/** + * An optional mix-in for columnar [[FileFormat]]s. This trait provides some helpful metadata when + * debugging a physical query plan. + */ +private[sql] trait ColumnarFileFormat { + _: FileFormat => + + /** Returns the number of columns in this file format required to satisfy the given schema. */ + def columnCountForSchema(sparkSession: SparkSession, readSchema: StructType): Int +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index b2409f3470e73..6610795cfdd21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -55,6 +55,7 @@ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} class ParquetFileFormat extends FileFormat + with ColumnarFileFormat with DataSourceRegister with Logging with Serializable { @@ -72,6 +73,14 @@ class ParquetFileFormat override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] + override def columnCountForSchema(sparkSession: SparkSession, readSchema: StructType): Int = { + val converter = new SparkToParquetSchemaConverter( + sparkSession.sessionState.conf.writeLegacyParquetFormat, + sparkSession.sessionState.conf.parquetOutputTimestampType) + val parquetSchema = converter.convert(readSchema) + parquetSchema.getPaths.size + } + override def prepareWrite( sparkSession: SparkSession, job: Job, @@ -423,11 +432,12 @@ class ParquetFileFormat } else { logDebug(s"Falling back to parquet-mr") // ParquetRecordReader returns UnsafeRow + val readSupport = new ParquetReadSupport(convertTz, true) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 40ce5d5e0564e..00db3cc62ff4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -47,16 +47,25 @@ import org.apache.spark.sql.types._ * * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. + * + * @param parquetMrCompatibility support reading with parquet-mr or Spark's built-in Parquet reader */ -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) +private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], + parquetMrCompatibility: Boolean) extends ReadSupport[UnsafeRow] with Logging { private var catalystRequestedSchema: StructType = _ + /** + * Construct a [[ParquetReadSupport]] with [[convertTz]] set to [[None]] and + * [[parquetMrCompatibility]] set to [[false]]. + * + * We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only + * used in the vectorized reader, where we get the convertTz value directly, and the value here + * is ignored. Further, we set [[parquetMrCompatibility]] to [[false]] as this constructor is only + * called by the Spark reader. + */ def this() { - // We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only - // used in the vectorized reader, where we get the convertTz value directly, and the value here - // is ignored. - this(None) + this(None, false) } /** @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) StructType.fromString(schemaString) } - val parquetRequestedSchema = + val clippedParquetSchema = ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema) + val parquetRequestedSchema = if (parquetMrCompatibility) { + // Parquet-mr will throw an exception if we try to read a superset of the file's schema. + // Therefore, we intersect our clipped schema with the underlying file's schema + ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema) + .map(intersectionGroup => + new MessageType(intersectionGroup.getName, intersectionGroup.getFields)) + .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE) + } else { + // Spark's built-in Parquet reader will throw an exception in some cases if the requested + // schema is not the same as the clipped schema + clippedParquetSchema + } + new ReadContext(parquetRequestedSchema, Map.empty[String, String].asJava) } @@ -96,7 +118,7 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) |Parquet form: |$parquetRequestedSchema |Catalyst form: - |$catalystRequestedSchema + |${catalystRequestedSchema.prettyJson} """.stripMargin } @@ -288,6 +310,27 @@ private[parquet] object ParquetReadSupport { } } + /** + * Computes the structural intersection between two Parquet group types. + */ + private def intersectParquetGroups( + groupType1: GroupType, groupType2: GroupType): Option[GroupType] = { + val fields = + groupType1.getFields.asScala + .filter(field => groupType2.containsField(field.getName)) + .flatMap { + case field1: GroupType => + intersectParquetGroups(field1, groupType2.getType(field1.getName).asGroupType) + case field1 => Some(field1) + } + + if (fields.nonEmpty) { + Some(groupType1.withNewFields(fields.asJava)) + } else { + None + } + } + def expandUDT(schema: StructType): StructType = { def expand(dataType: DataType): DataType = { dataType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 1199725941842..8bf0f32fdf619 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -130,8 +130,8 @@ private[parquet] class ParquetRowConverter( extends ParquetGroupConverter(updater) with Logging { assert( - parquetType.getFieldCount == catalystType.length, - s"""Field counts of the Parquet schema and the Catalyst schema don't match: + parquetType.getFieldCount <= catalystType.length, + s"""Field count of the Parquet schema is greater than the field count of the Catalyst schema: | |Parquet schema: |$parquetType @@ -182,10 +182,12 @@ private[parquet] class ParquetRowConverter( // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { - parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map { - case ((parquetFieldType, catalystField), ordinal) => - // Converted field value should be set to the `ordinal`-th cell of `currentRow` - newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) + parquetType.getFields.asScala.map { + case parquetField => + val fieldIndex = catalystType.fieldIndex(parquetField.getName) + val catalystField = catalystType(fieldIndex) + // Converted field value should be set to the `fieldIndex`-th cell of `currentRow` + newConverter(parquetField, catalystField.dataType, new RowUpdater(currentRow, fieldIndex)) }.toArray } @@ -193,7 +195,7 @@ private[parquet] class ParquetRowConverter( override def end(): Unit = { var i = 0 - while (i < currentRow.numFields) { + while (i < fieldConverters.length) { fieldConverters(i).updater.end() i += 1 } @@ -202,11 +204,15 @@ private[parquet] class ParquetRowConverter( override def start(): Unit = { var i = 0 - while (i < currentRow.numFields) { + while (i < fieldConverters.length) { fieldConverters(i).updater.start() currentRow.setNullAt(i) i += 1 } + while (i < currentRow.numFields) { + currentRow.setNullAt(i) + i += 1 + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala new file mode 100644 index 0000000000000..c66afff2bc70f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -0,0 +1,154 @@ +/* + * 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.execution.datasources.parquet + +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} + +/** + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a + * SQL column, and a nested Parquet column corresponds to a [[StructField]]. + */ +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = + if (SQLConf.get.nestedSchemaPruningEnabled) { + apply0(plan) + } else { + plan + } + + private def apply0(plan: LogicalPlan): LogicalPlan = + plan transformDown { + case op @ PhysicalOperation(projects, filters, + l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema, + dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) => + val projectionFields = projects.flatMap(getFields) + val filterFields = filters.flatMap(getFields) + val requestedFields = (projectionFields ++ filterFields).distinct + + // If [[requestedFields]] includes a nested field, continue. Otherwise, + // return [[op]] + if (requestedFields.exists { case (_, optAtt) => optAtt.isEmpty }) { + val prunedSchema = requestedFields + .map { case (field, _) => StructType(Array(field)) } + .reduceLeft(_ merge _) + val dataSchemaFieldNames = dataSchema.fieldNames.toSet + val prunedDataSchema = + StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) + + // If the data schema is different from the pruned data schema, continue. Otherwise, + // return [[op]]. We effect this comparison by counting the number of "leaf" fields in + // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields + // in [[dataSchema]]. + if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) { + val prunedParquetRelation = + hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession) + + // We need to replace the expression ids of the pruned relation output attributes + // with the expression ids of the original relation output attributes so that + // references to the original relation's output are not broken + val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap + val prunedRelationOutput = + prunedParquetRelation + .schema + .toAttributes + .map { + case att if outputIdMap.contains(att.name) => + att.withExprId(outputIdMap(att.name)) + case att => att + } + val prunedRelation = + l.copy(relation = prunedParquetRelation, output = prunedRelationOutput) + + val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) + + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + Filter(newFilterCondition, prunedRelation) + } else { + prunedRelation + } + + val nonDataPartitionColumnNames = + partitionSchema.map(_.name).filterNot(dataSchemaFieldNames.contains).toSet + + // Construct the new projections of our [[Project]] by + // rewriting the original projections + val newProjects = projects.map { + case project if (nonDataPartitionColumnNames.contains(project.name)) => project + case project => + (project transformDown { + case projectionOverSchema(expr) => expr + }).asInstanceOf[NamedExpression] + } + + logDebug("New projects:\n" + newProjects.map(_.treeString).mkString("\n")) + logDebug(s"Pruned data schema:\n${prunedDataSchema.treeString}") + + Project(newProjects, projectionChild) + } else { + op + } + } else { + op + } + } + + /** + * Gets the top-level (no-parent) [[StructField]]s for the given [[Expression]]. + * When [[expr]] is an [[Attribute]], construct a field around it and return the + * attribute as the second component of the returned tuple. + */ + private def getFields(expr: Expression): Seq[(StructField, Option[Attribute])] = { + expr match { + case att: Attribute => + (StructField(att.name, att.dataType, att.nullable), Some(att)) :: Nil + case SelectedField(field) => (field, None) :: Nil + case _ => + expr.children.flatMap(getFields) + } + } + + /** + * Counts the "leaf" fields of the given [[dataType]]. Informally, this is the + * number of fields of non-complex data type in the tree representation of + * [[dataType]]. + */ + private def countLeaves(dataType: DataType): Int = { + dataType match { + case array: ArrayType => countLeaves(array.elementType) + case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) + case struct: StructType => + struct.map(field => countLeaves(field.dataType)).sum + case _ => 1 + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b0e22a51e7611..430101b16459c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -2499,6 +2499,21 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df, df.collect()) } + test("SPARK-4502: Nested column pruning shouldn't fail filter") { + withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + val data = + """{"a":{"b":1,"c":2}} + |{}""".stripMargin + Seq(data).toDF().repartition(1).write.text(path) + checkAnswer( + spark.read.json(path).filter($"a.b" > 1).select($"a.b"), + Seq.empty) + } + } + } + test("SPARK-24313: access map with binary keys") { val mapWithBinaryKey = map(lit(Array[Byte](1.toByte)), lit(1)) checkAnswer(spark.range(1).select(mapWithBinaryKey.getItem(Array[Byte](1.toByte))), Row(1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala new file mode 100644 index 0000000000000..e7b634d227c85 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala @@ -0,0 +1,54 @@ +/* + * 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.execution + +import org.scalactic.Equality +import org.scalatest.Assertions + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.SchemaPruningTest +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.types.StructType + +private[sql] trait FileSchemaPruningTest extends SchemaPruningTest { + _: Assertions => + + private val schemaEquality = new Equality[StructType] { + override def areEqual(a: StructType, b: Any) = + b match { + case otherType: StructType => a sameType otherType + case _ => false + } + } + + protected def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan.requiredSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected ${expectedSchemaCatalogStrings}") + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index dbf637783e6d2..72cccb9ae519a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -108,7 +108,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext val queryOutput = selfJoin.queryExecution.analyzed.output assertResult(4, "Field count mismatches")(queryOutput.size) - assertResult(2, "Duplicated expression ID in query plan:\n $selfJoin") { + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { queryOutput.filter(_.name == "_1").map(_.exprId).size } @@ -117,7 +117,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } test("nested data - struct with array field") { - val data = (1 to 10).map(i => Tuple1((i, Seq("val_$i")))) + val data = (1 to 10).map(i => Tuple1((i, Seq(s"val_$i")))) withParquetTable(data, "t") { checkAnswer(sql("SELECT _1._2[0] FROM t"), data.map { case Tuple1((_, Seq(string))) => Row(string) @@ -126,7 +126,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } test("nested data - array of struct") { - val data = (1 to 10).map(i => Tuple1(Seq(i -> "val_$i"))) + val data = (1 to 10).map(i => Tuple1(Seq(i -> s"val_$i"))) withParquetTable(data, "t") { checkAnswer(sql("SELECT _1[0]._2 FROM t"), data.map { case Tuple1(Seq((_, string))) => Row(string) @@ -880,6 +880,15 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } + test("select function over nested data") { + val data = (1 to 10).map(i => Tuple1((i, s"val_$i"))) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT isnotnull(_1._2) FROM t"), data.map { + case _ => Row(true) + }) + } + } + test("SPARK-24230: filter row group using dictionary") { withSQLConf(("parquet.filter.dictionary.enabled", "true")) { // create a table with values from 0, 2, ..., 18 that will be dictionary-encoded diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala new file mode 100644 index 0000000000000..4bc7a553b23b4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -0,0 +1,124 @@ +/* + * 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.execution.datasources.parquet + +import java.io.File + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.FileSchemaPruningTest +import org.apache.spark.sql.test.SharedSQLContext + +class ParquetSchemaPruningSuite + extends QueryTest + with ParquetTest + with FileSchemaPruningTest + with SharedSQLContext { + case class FullName(first: String, middle: String, last: String) + case class Contact(name: FullName, address: String, pets: Int, friends: Array[FullName] = Array(), + relatives: Map[String, FullName] = Map()) + + val contacts = + Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) :: + Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil + + case class Name(first: String, last: String) + case class BriefContact(name: Name, address: String) + + val briefContacts = + BriefContact(Name("Janet", "Jones"), "567 Maple Drive") :: + BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil + + testStandardAndLegacyModes("partial schema intersection - select missing subfield") { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeParquetFile(contacts, new File(path + "/contacts/p=1")) + makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) + + spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + + val query = sql("select name.middle, address from contacts where p=2") + checkScanSchemata(query, "struct,address:string>") + checkAnswer(query, + Row(null, "567 Maple Drive") :: + Row(null, "6242 Ash Street") :: Nil) + } + } + + testStandardAndLegacyModes("partial schema intersection - filter on subfield") { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeParquetFile(contacts, new File(path + "/contacts/p=1")) + makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) + + spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + + val query = + sql("select name.middle, name.first, pets, address from contacts where " + + "name.first = 'Janet' and p=2") + checkScanSchemata(query, + "struct,pets:int,address:string>") + checkAnswer(query, + Row(null, "Janet", null, "567 Maple Drive") :: Nil) + } + } + + testStandardAndLegacyModes("no unnecessary schema pruning") { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeParquetFile(contacts, new File(path + "/contacts/p=1")) + makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) + + spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + + val query = + sql("select name.last, name.middle, name.first, relatives[''].last, " + + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + + "friends[0].first, pets, address from contacts where p=2") + // We've selected every field in the schema. Therefore, no schema pruning should be performed. + // We check this by asserting that the scanned schema of the query is identical to the schema + // of the contacts relation, even though the fields are selected in different orders. + checkScanSchemata(query, + "struct,address:string,pets:int," + + "friends:array>," + + "relatives:map>>") + checkAnswer(query, + Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: + Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: + Nil) + } + } + + testStandardAndLegacyModes("empty schema intersection") { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeParquetFile(contacts, new File(path + "/contacts/p=1")) + makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) + + spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + + val query = sql("select name.middle from contacts where p=2") + checkScanSchemata(query, "struct>") + checkAnswer(query, + Row(null) :: Row(null) :: Nil) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index f05f5722af51a..bf15525cbda34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -59,42 +59,68 @@ private[sql] trait ParquetTest extends SQLTestUtils { } /** - * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` + * Writes `df` to a Parquet file, which is then passed to `f` and will be deleted after `f` * returns. */ - protected def withParquetFile[T <: Product: ClassTag: TypeTag] - (data: Seq[T]) + protected def asParquetFile + (df: DataFrame) (f: String => Unit): Unit = { withTempPath { file => - spark.createDataFrame(data).write.parquet(file.getCanonicalPath) + df.write.parquet(file.getCanonicalPath) f(file.getCanonicalPath) } } + /** + * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withParquetFile[T <: Product: ClassTag: TypeTag] + (data: Seq[T]) + (f: String => Unit): Unit = asParquetFile(spark.createDataFrame(data))(f) + + /** + * Writes `df` to a Parquet file and reads it back as a [[DataFrame]], + * which is then passed to `f`. The Parquet file will be deleted after `f` returns. + */ + protected def asParquetDataFrame + (df: DataFrame, testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = + asParquetFile(df)(path => readParquetFile(path.toString, testVectorized)(f)) + /** * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] (data: Seq[T], testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = { - withParquetFile(data)(path => readParquetFile(path.toString, testVectorized)(f)) - } + (f: DataFrame => Unit): Unit = + asParquetDataFrame(spark.createDataFrame(data), testVectorized)(f) /** - * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a + * Writes `df` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a * temporary table named `tableName`, then call `f`. The temporary table together with the * Parquet file will be dropped/deleted after `f` returns. */ - protected def withParquetTable[T <: Product: ClassTag: TypeTag] - (data: Seq[T], tableName: String, testVectorized: Boolean = true) + protected def asParquetTable + (df: DataFrame, tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { - withParquetDataFrame(data, testVectorized) { df => + asParquetDataFrame(df, testVectorized) { df => df.createOrReplaceTempView(tableName) withTempView(tableName)(f) } } + /** + * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a + * temporary table named `tableName`, then call `f`. The temporary table together with the + * Parquet file will be dropped/deleted after `f` returns. + */ + protected def withParquetTable[T <: Product: ClassTag: TypeTag] + (data: Seq[T], tableName: String, testVectorized: Boolean = true) + (f: => Unit): Unit = + asParquetTable(spark.createDataFrame(data), tableName, testVectorized)(f) + protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( data: Seq[T], path: File): Unit = { spark.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) From 0d0e8a077ef5e2e36033fe07b72d9711881e2b47 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jun 2018 17:59:34 +0800 Subject: [PATCH 02/24] Refactor SelectedFieldSuite to make its tests simpler and more comprehensible --- .../planning/SelectedFieldSuite.scala | 291 ++++++++---------- 1 file changed, 123 insertions(+), 168 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala index 6923c7a89abf9..ce1bda5301f78 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala @@ -178,236 +178,189 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { private val testRelation = LocalRelation(schema.toAttributes) - test("should not match an attribute reference") { + test("SelectedField should not match an attribute reference") { assertResult(None)(unapplySelect("col1")) assertResult(None)(unapplySelect("col1 as foo")) assertResult(None)(unapplySelect("col2")) } - test("col2.field2, col2.field2[0] as foo") { - val expected = - StructField("col2", StructType( - StructField("field2", ArrayType(IntegerType, containsNull = false)) :: Nil)) - testSelect("col2.field2", expected) - testSelect("col2.field2[0] as foo", expected) + info("For a relation with schema\n" + indent(schema.treeString)) + + testSelect("col2.field2", "col2.field2[0] as foo") { + StructField("col2", StructType( + StructField("field2", ArrayType(IntegerType, containsNull = false)) :: Nil)) } - test("col2.field9, col2.field9['foo'] as foo") { - val expected = - StructField("col2", StructType( - StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) - testSelect("col2.field9", expected) - testSelect("col2.field9['foo'] as foo", expected) + testSelect("col2.field9", "col2.field9['foo'] as foo") { + StructField("col2", StructType( + StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) } - test("col2.field3.subfield3, col2.field3[0].subfield3 as foo, col2.field3.subfield3[0] as foo, col2.field3[0].subfield3[0] as foo") { - val expected = - StructField("col2", StructType( - StructField("field3", ArrayType(StructType( - StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: Nil)) - testSelect("col2.field3.subfield3", expected) - testSelect("col2.field3[0].subfield3 as foo", expected) - testSelect("col2.field3.subfield3[0] as foo", expected) - testSelect("col2.field3[0].subfield3[0] as foo", expected) + testSelect("col2.field3.subfield3", "col2.field3[0].subfield3 as foo", + "col2.field3.subfield3[0] as foo", "col2.field3[0].subfield3[0] as foo") { + StructField("col2", StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: Nil)) } - test("col2.field3.subfield1") { - val expected = - StructField("col2", StructType( - StructField("field3", ArrayType(StructType( - StructField("subfield1", IntegerType) :: Nil)), nullable = false) :: Nil)) - testSelect("col2.field3.subfield1", expected) + testSelect("col2.field3.subfield1") { + StructField("col2", StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType) :: Nil)), nullable = false) :: Nil)) } - test("col2.field5.subfield1") { - val expected = - StructField("col2", StructType( - StructField("field5", ArrayType(StructType( - StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: - StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) - testSelect("col2.field5.subfield1", expected) + testSelect("col2.field5.subfield1") { + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) } - test("col3.field1.subfield1") { - val expected = - StructField("col3", ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), containsNull = false), nullable = false) - testSelect("col3.field1.subfield1", expected) + testSelect("col3.field1.subfield1") { + StructField("col3", ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), containsNull = false), nullable = false) } - test("col3.field2['foo'] as foo") { - val expected = - StructField("col3", ArrayType(StructType( - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) - testSelect("col3.field2['foo'] as foo", expected) + testSelect("col3.field2['foo'] as foo") { + StructField("col3", ArrayType(StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) } - test("col4['foo'].field1.subfield1 as foo") { - val expected = - StructField("col4", MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), valueContainsNull = false), nullable = false) - testSelect("col4['foo'].field1.subfield1 as foo", expected) + testSelect("col4['foo'].field1.subfield1 as foo") { + StructField("col4", MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), valueContainsNull = false), nullable = false) } - test("col4['foo'].field2['bar'] as foo") { - val expected = - StructField("col4", MapType(StringType, StructType( - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) - testSelect("col4['foo'].field2['bar'] as foo", expected) + testSelect("col4['foo'].field2['bar'] as foo") { + StructField("col4", MapType(StringType, StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) } - test("col5[0]['foo'].field1.subfield1 as foo") { - val expected = - StructField("col5", ArrayType(MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) - testSelect("col5[0]['foo'].field1.subfield1 as foo", expected) + testSelect("col5[0]['foo'].field1.subfield1 as foo") { + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) } - test("col6['foo'][0].field1.subfield1 as foo") { - val expected = - StructField("col6", MapType(StringType, ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false))) - testSelect("col6['foo'][0].field1.subfield1 as foo", expected) + testSelect("col6['foo'][0].field1.subfield1 as foo") { + StructField("col6", MapType(StringType, ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false))) } - test("col2.field5.subfield1.subsubfield1") { - val expected = - StructField("col2", StructType( - StructField("field5", ArrayType(StructType( - StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) - testSelect("col2.field5.subfield1.subsubfield1", expected) + testSelect("col2.field5.subfield1.subsubfield1") { + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) } - test("col2.field5.subfield2.subsubfield1.subsubsubfield1") { - val expected = - StructField("col2", StructType( - StructField("field5", ArrayType(StructType( - StructField("subfield2", StructType( - StructField("subsubfield1", StructType( - StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) :: Nil)), nullable = false) :: Nil)) - testSelect("col2.field5.subfield2.subsubfield1.subsubsubfield1", expected) + testSelect("col2.field5.subfield2.subsubfield1.subsubsubfield1") { + StructField("col2", StructType( + StructField("field5", ArrayType(StructType( + StructField("subfield2", StructType( + StructField("subsubfield1", StructType( + StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) :: Nil)), nullable = false) :: Nil)) } - test("col2.field4['foo'].subfield1 as foo") { - val expected = - StructField("col2", StructType( - StructField("field4", MapType(StringType, StructType( - StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil)) - testSelect("col2.field4['foo'].subfield1 as foo", expected) + testSelect("col2.field4['foo'].subfield1 as foo") { + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil)) } - test("col2.field4['foo'].subfield2 as foo, col2.field4['foo'].subfield2[0] as foo") { - val expected = - StructField("col2", StructType( - StructField("field4", MapType(StringType, StructType( - StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: Nil)) - testSelect("col2.field4['foo'].subfield2 as foo", expected) - testSelect("col2.field4['foo'].subfield2[0] as foo", expected) + testSelect("col2.field4['foo'].subfield2 as foo", "col2.field4['foo'].subfield2[0] as foo") { + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: Nil)) } - test("col2.field8['foo'][0].subfield1 as foo") { - val expected = - StructField("col2", StructType( - StructField("field8", MapType(StringType, ArrayType(StructType( - StructField("subfield1", IntegerType) :: Nil)), valueContainsNull = false)) :: Nil)) - testSelect("col2.field8['foo'][0].subfield1 as foo", expected) + testSelect("col2.field8['foo'][0].subfield1 as foo") { + StructField("col2", StructType( + StructField("field8", MapType(StringType, ArrayType(StructType( + StructField("subfield1", IntegerType) :: Nil)), valueContainsNull = false)) :: Nil)) } - test("col2.field1") { - val expected = - StructField("col2", StructType( - StructField("field1", IntegerType) :: Nil)) - testSelect("col2.field1", expected) + testSelect("col2.field1") { + StructField("col2", StructType( + StructField("field1", IntegerType) :: Nil)) } - test("col2.field6") { - val expected = - StructField("col2", StructType( - StructField("field6", StructType( - StructField("subfield1", StringType, nullable = false) :: - StructField("subfield2", StringType) :: Nil)) :: Nil)) - testSelect("col2.field6", expected) + testSelect("col2.field6") { + StructField("col2", StructType( + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: + StructField("subfield2", StringType) :: Nil)) :: Nil)) } - test("col2.field7.subfield1") { - val expected = - StructField("col2", StructType( - StructField("field7", StructType( - StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: - StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: Nil)) - testSelect("col2.field7.subfield1", expected) + testSelect("col2.field7.subfield1") { + StructField("col2", StructType( + StructField("field7", StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: Nil)) } - test("col2.field6.subfield1") { - val expected = - StructField("col2", StructType( - StructField("field6", StructType( - StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil)) - testSelect("col2.field6.subfield1", expected) + testSelect("col2.field6.subfield1") { + StructField("col2", StructType( + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil)) } - test("col7.field1, col7[0].field1 as foo, col7.field1[0] as foo") { - val expected = - StructField("col7", ArrayType(StructType( - StructField("field1", IntegerType, nullable = false) :: Nil))) - testSelect("col7.field1", expected) - testSelect("col7[0].field1 as foo", expected) - testSelect("col7.field1[0] as foo", expected) + testSelect("col7.field1", "col7[0].field1 as foo", "col7.field1[0] as foo") { + StructField("col7", ArrayType(StructType( + StructField("field1", IntegerType, nullable = false) :: Nil))) } - test("col7.field2.subfield1") { - val expected = - StructField("col7", ArrayType(StructType( - StructField("field2", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil))) - testSelect("col7.field2.subfield1", expected) + testSelect("col7.field2.subfield1") { + StructField("col7", ArrayType(StructType( + StructField("field2", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil))) } - test("col7.field3.subfield1") { - val expected = - StructField("col7", ArrayType(StructType( - StructField("field3", ArrayType(StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) - testSelect("col7.field3.subfield1", expected) + testSelect("col7.field3.subfield1") { + StructField("col7", ArrayType(StructType( + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) } - test("col8.field1, col8[0].field1 as foo, col8.field1[0] as foo, col8[0].field1[0] as foo") { - val expected = - StructField("col8", ArrayType(StructType( - StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) - testSelect("col8.field1", expected) - testSelect("col8[0].field1 as foo", expected) - testSelect("col8.field1[0] as foo", expected) - testSelect("col8[0].field1[0] as foo", expected) + testSelect("col8.field1", "col8[0].field1 as foo", "col8.field1[0] as foo", "col8[0].field1[0] as foo") { + StructField("col8", ArrayType(StructType( + StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) } - def assertResult(expected: StructField)(actual: StructField)(expr: String): Unit = { + def assertResult(expected: StructField)(actual: StructField)(selectExpr: String): Unit = { try { super.assertResult(expected)(actual) } catch { case ex: TestFailedException => // Print some helpful diagnostics in the case of failure - // scalastyle:off println - println("For " + expr) - println("Expected:") - println(StructType(expected :: Nil).treeString) - println("Actual:") - println(StructType(actual :: Nil).treeString) - println("expected.dataType.sameType(actual.dataType) = " + - expected.dataType.sameType(actual.dataType)) - // scalastyle:on println + alert("Expected SELECT \"" + selectExpr + "\" to select the schema\n" + + indent(StructType(expected :: Nil).treeString) + + indent("but it actually selected\n") + + indent(StructType(actual :: Nil).treeString) + + indent("Note that expected.dataType.sameType(actual.dataType) = " + + expected.dataType.sameType(actual.dataType))) throw ex } } - private def testSelect(expr: String, expected: StructField) = { + // Test that the given SELECT expressions prune the test schema to the single-column schema + // defined by the given field + private def testSelect(selectExpr: String, otherSelectExprs: String*)(expected: StructField) { + val selectExprs = selectExpr +: otherSelectExprs + test(s"SELECT " + selectExprs.map(s => s""""$s"""").mkString(", ") + " should select the schema\n" + + indent(StructType(expected :: Nil).treeString)) { + for (selectExpr <- selectExprs) { + assertSelect(selectExpr, expected) + } + } + } + + private def assertSelect(expr: String, expected: StructField) = { unapplySelect(expr) match { case Some(field) => assertResult(expected)(field)(expr) @@ -429,4 +382,6 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { val analyzed = select.analyze SelectedField.unapply(analyzed.expressions.head) } + + private def indent(string: String) = string.replaceAll("(?m)^", " ") } From 44e78cbf3bf670c0de9d3efd6bd05c4d44628575 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jun 2018 18:02:28 +0800 Subject: [PATCH 03/24] Remove test "select function over nested data" of unknown origin and purpose --- .../datasources/parquet/ParquetQuerySuite.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 72cccb9ae519a..54c77dddc3525 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -880,15 +880,6 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } - test("select function over nested data") { - val data = (1 to 10).map(i => Tuple1((i, s"val_$i"))) - withParquetTable(data, "t") { - checkAnswer(sql("SELECT isnotnull(_1._2) FROM t"), data.map { - case _ => Row(true) - }) - } - } - test("SPARK-24230: filter row group using dictionary") { withSQLConf(("parquet.filter.dictionary.enabled", "true")) { // create a table with values from 0, 2, ..., 18 that will be dictionary-encoded From 9488cb5c9d33670bd05f14ed00a24e68ae79f2ea Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jun 2018 18:45:28 +0800 Subject: [PATCH 04/24] Improve readability of ParquetSchemaPruning and ParquetSchemaPruningSuite. Add test to exercise whether the requested root fields in a query exclude any attributes --- .../parquet/ParquetSchemaPruning.scala | 34 +++++++------ .../parquet/ParquetSchemaPruningSuite.scala | 49 ++++++++----------- 2 files changed, 41 insertions(+), 42 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index c66afff2bc70f..c54c8631bc786 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -44,15 +44,15 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { case op @ PhysicalOperation(projects, filters, l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema, dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) => - val projectionFields = projects.flatMap(getFields) - val filterFields = filters.flatMap(getFields) - val requestedFields = (projectionFields ++ filterFields).distinct + val projectionRootFields = projects.flatMap(getRootFields) + val filterRootFields = filters.flatMap(getRootFields) + val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct - // If [[requestedFields]] includes a nested field, continue. Otherwise, + // If [[requestedRootFields]] includes a nested field, continue. Otherwise, // return [[op]] - if (requestedFields.exists { case (_, optAtt) => optAtt.isEmpty }) { - val prunedSchema = requestedFields - .map { case (field, _) => StructType(Array(field)) } + if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) { + val prunedSchema = requestedRootFields + .map { case RootField(field, _) => StructType(Array(field)) } .reduceLeft(_ merge _) val dataSchemaFieldNames = dataSchema.fieldNames.toSet val prunedDataSchema = @@ -123,17 +123,17 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { } /** - * Gets the top-level (no-parent) [[StructField]]s for the given [[Expression]]. - * When [[expr]] is an [[Attribute]], construct a field around it and return the - * attribute as the second component of the returned tuple. + * Gets the root (aka top-level, no-parent) [[StructField]]s for the given [[Expression]]. + * When [[expr]] is an [[Attribute]], construct a field around it and indicate that that + * field was derived from an attribute. */ - private def getFields(expr: Expression): Seq[(StructField, Option[Attribute])] = { + private def getRootFields(expr: Expression): Seq[RootField] = { expr match { case att: Attribute => - (StructField(att.name, att.dataType, att.nullable), Some(att)) :: Nil - case SelectedField(field) => (field, None) :: Nil + RootField(StructField(att.name, att.dataType, att.nullable), true) :: Nil + case SelectedField(field) => RootField(field, false) :: Nil case _ => - expr.children.flatMap(getFields) + expr.children.flatMap(getRootFields) } } @@ -151,4 +151,10 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { case _ => 1 } } + + /** + * A "root" schema field (aka top-level, no-parent) and whether it was derived from + * an attribute or had a proper child. + */ + private case class RootField(field: StructField, derivedFromAtt: Boolean) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 4bc7a553b23b4..5bd68057400de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -43,15 +43,16 @@ class ParquetSchemaPruningSuite BriefContact(Name("Janet", "Jones"), "567 Maple Drive") :: BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil - testStandardAndLegacyModes("partial schema intersection - select missing subfield") { - withTempPath { dir => - val path = dir.getCanonicalPath - - makeParquetFile(contacts, new File(path + "/contacts/p=1")) - makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) - - spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + testStandardAndLegacyModes("prune a single field") { + withContacts { + val query = sql("select name.middle from contacts") + checkScanSchemata(query, "struct>") + checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) + } + } + testStandardAndLegacyModes("partial schema intersection - select missing subfield") { + withContacts { val query = sql("select name.middle, address from contacts where p=2") checkScanSchemata(query, "struct,address:string>") checkAnswer(query, @@ -61,14 +62,7 @@ class ParquetSchemaPruningSuite } testStandardAndLegacyModes("partial schema intersection - filter on subfield") { - withTempPath { dir => - val path = dir.getCanonicalPath - - makeParquetFile(contacts, new File(path + "/contacts/p=1")) - makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) - - spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") - + withContacts { val query = sql("select name.middle, name.first, pets, address from contacts where " + "name.first = 'Janet' and p=2") @@ -80,14 +74,7 @@ class ParquetSchemaPruningSuite } testStandardAndLegacyModes("no unnecessary schema pruning") { - withTempPath { dir => - val path = dir.getCanonicalPath - - makeParquetFile(contacts, new File(path + "/contacts/p=1")) - makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) - - spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") - + withContacts { val query = sql("select name.last, name.middle, name.first, relatives[''].last, " + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + @@ -107,6 +94,15 @@ class ParquetSchemaPruningSuite } testStandardAndLegacyModes("empty schema intersection") { + withContacts { + val query = sql("select name.middle from contacts where p=2") + checkScanSchemata(query, "struct>") + checkAnswer(query, + Row(null) :: Row(null) :: Nil) + } + } + + private def withContacts(testThunk: => Unit) { withTempPath { dir => val path = dir.getCanonicalPath @@ -115,10 +111,7 @@ class ParquetSchemaPruningSuite spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") - val query = sql("select name.middle from contacts where p=2") - checkScanSchemata(query, "struct>") - checkAnswer(query, - Row(null) :: Row(null) :: Nil) + testThunk } } } From f3735b1945b141051f726ac540e8fd8938bd6189 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jun 2018 20:04:23 +0800 Subject: [PATCH 05/24] Don't handle non-data-field partition column names specially when constructing the new projections in ParquetSchemaPruning. These column projections are left unchanged by the transformDown function when the projectionOverSchema pattern does not match --- .../parquet/ParquetSchemaPruning.scala | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index c54c8631bc786..f31d24a549223 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -42,7 +42,7 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { private def apply0(plan: LogicalPlan): LogicalPlan = plan transformDown { case op @ PhysicalOperation(projects, filters, - l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema, + l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _, dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) => val projectionRootFields = projects.flatMap(getRootFields) val filterRootFields = filters.flatMap(getRootFields) @@ -97,20 +97,13 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { prunedRelation } - val nonDataPartitionColumnNames = - partitionSchema.map(_.name).filterNot(dataSchemaFieldNames.contains).toSet - // Construct the new projections of our [[Project]] by // rewriting the original projections - val newProjects = projects.map { - case project if (nonDataPartitionColumnNames.contains(project.name)) => project - case project => - (project transformDown { - case projectionOverSchema(expr) => expr - }).asInstanceOf[NamedExpression] - } + val newProjects = projects.map(_.transformDown { + case projectionOverSchema(expr) => expr + }).map { case expr: NamedExpression => expr } - logDebug("New projects:\n" + newProjects.map(_.treeString).mkString("\n")) + logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") logDebug(s"Pruned data schema:\n${prunedDataSchema.treeString}") Project(newProjects, projectionChild) From 2120ab55322633495e52fb676bc71a004d9598ea Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jun 2018 20:07:35 +0800 Subject: [PATCH 06/24] Add test coverage for ParquetSchemaPruning for partitioned tables whose data schema includes the partition column --- .../parquet/ParquetSchemaPruningSuite.scala | 145 ++++++++++++------ 1 file changed, 97 insertions(+), 48 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 5bd68057400de..b7f4858f4912c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -21,6 +21,7 @@ import java.io.File import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.FileSchemaPruningTest +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class ParquetSchemaPruningSuite @@ -43,65 +44,100 @@ class ParquetSchemaPruningSuite BriefContact(Name("Janet", "Jones"), "567 Maple Drive") :: BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil - testStandardAndLegacyModes("prune a single field") { - withContacts { - val query = sql("select name.middle from contacts") - checkScanSchemata(query, "struct>") - checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) - } + case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int, + friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), p: Int) + + case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int) + + val contactsWithDataPartitionColumn = + contacts.map { case Contact(name, address, pets, friends, relatives) => + ContactWithDataPartitionColumn(name, address, pets, friends, relatives, 1) } + val briefContactsWithDataPartitionColumn = + briefContacts.map { case BriefContact(name: Name, address: String) => + BriefContactWithDataPartitionColumn(name, address, 2) } + + testSchemaPruning("select a single complex field") { + val query = sql("select name.middle from contacts") + checkScanSchemata(query, "struct>") + checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) } - testStandardAndLegacyModes("partial schema intersection - select missing subfield") { - withContacts { - val query = sql("select name.middle, address from contacts where p=2") - checkScanSchemata(query, "struct,address:string>") - checkAnswer(query, - Row(null, "567 Maple Drive") :: - Row(null, "6242 Ash Street") :: Nil) - } + testSchemaPruning("select a single complex field and the partition column") { + val query = sql("select name.middle, p from contacts") + checkScanSchemata(query, "struct>") + checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - testStandardAndLegacyModes("partial schema intersection - filter on subfield") { - withContacts { - val query = - sql("select name.middle, name.first, pets, address from contacts where " + - "name.first = 'Janet' and p=2") - checkScanSchemata(query, - "struct,pets:int,address:string>") - checkAnswer(query, - Row(null, "Janet", null, "567 Maple Drive") :: Nil) - } + testSchemaPruning("partial schema intersection - select missing subfield") { + val query = sql("select name.middle, address from contacts where p=2") + checkScanSchemata(query, "struct,address:string>") + checkAnswer(query, + Row(null, "567 Maple Drive") :: + Row(null, "6242 Ash Street") :: Nil) } - testStandardAndLegacyModes("no unnecessary schema pruning") { - withContacts { - val query = - sql("select name.last, name.middle, name.first, relatives[''].last, " + - "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + - "friends[0].first, pets, address from contacts where p=2") - // We've selected every field in the schema. Therefore, no schema pruning should be performed. - // We check this by asserting that the scanned schema of the query is identical to the schema - // of the contacts relation, even though the fields are selected in different orders. - checkScanSchemata(query, - "struct,address:string,pets:int," + - "friends:array>," + - "relatives:map>>") - checkAnswer(query, - Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: - Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: - Nil) - } + testSchemaPruning("partial schema intersection - filter on subfield") { + val query = + sql("select name.middle, name.first, pets, address from contacts where " + + "name.first = 'Janet' and p=2") + checkScanSchemata(query, + "struct,pets:int,address:string>") + checkAnswer(query, + Row(null, "Janet", null, "567 Maple Drive") :: Nil) } - testStandardAndLegacyModes("empty schema intersection") { - withContacts { - val query = sql("select name.middle from contacts where p=2") - checkScanSchemata(query, "struct>") - checkAnswer(query, - Row(null) :: Row(null) :: Nil) + testSchemaPruning("no unnecessary schema pruning") { + val query = + sql("select name.last, name.middle, name.first, relatives[''].last, " + + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + + "friends[0].first, pets, address from contacts where p=2") + // We've selected every field in the schema. Therefore, no schema pruning should be performed. + // We check this by asserting that the scanned schema of the query is identical to the schema + // of the contacts relation, even though the fields are selected in different orders. + checkScanSchemata(query, + "struct,address:string,pets:int," + + "friends:array>," + + "relatives:map>>") + checkAnswer(query, + Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: + Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: + Nil) + } + + testSchemaPruning("empty schema intersection") { + val query = sql("select name.middle from contacts where p=2") + checkScanSchemata(query, "struct>") + checkAnswer(query, + Row(null) :: Row(null) :: Nil) + } + + private def testSchemaPruning(testName: String)(testThunk: => Unit) { + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") { + test(s"Standard mode - without partition data column - $testName") { + withContacts(testThunk) + } + test(s"Standard mode - with partition data column - $testName") { + withContactsWithDataPartitionColumn(testThunk) + } + } + + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "true") { + test(s"Legacy mode - without partition data column - $testName") { + withContacts(testThunk) + } + test(s"Legacy mode - with partition data column - $testName") { + withContactsWithDataPartitionColumn(testThunk) + } } } + private def withContactTables(testThunk: => Unit) { + info("testing table without partition data column") + withContacts(testThunk) + info("testing table with partition data column") + withContactsWithDataPartitionColumn(testThunk) + } + private def withContacts(testThunk: => Unit) { withTempPath { dir => val path = dir.getCanonicalPath @@ -114,4 +150,17 @@ class ParquetSchemaPruningSuite testThunk } } + + private def withContactsWithDataPartitionColumn(testThunk: => Unit) { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeParquetFile(contactsWithDataPartitionColumn, new File(path + "/contacts/p=1")) + makeParquetFile(briefContactsWithDataPartitionColumn, new File(path + "/contacts/p=2")) + + spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") + + testThunk + } + } } From 8d53bbde33fd25adc6f74318c017c034c2cc6f66 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 11 Jun 2018 17:22:40 -0700 Subject: [PATCH 07/24] Remove the ColumnarFileFormat type to put it in another PR --- .../sql/execution/DataSourceScanExec.scala | 15 +-------- .../datasources/ColumnarFileFormat.scala | 32 ------------------- .../parquet/ParquetFileFormat.scala | 9 ------ 3 files changed, 1 insertion(+), 55 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 0bf7248b06e9b..40b8ebed607f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -303,20 +303,7 @@ case class FileSourceScanExec( } getOrElse { withOptPartitionCount } - - val withOptColumnCount = relation.fileFormat match { - case columnar: ColumnarFileFormat => - SparkSession - .getActiveSession - .map { sparkSession => - val columnCount = columnar.columnCountForSchema(sparkSession, requiredSchema) - withSelectedBucketsCount + ("ColumnCount" -> columnCount.toString) - } getOrElse { - withSelectedBucketsCount - } - case _ => withSelectedBucketsCount - } - withOptColumnCount + withSelectedBucketsCount } private lazy val inputRDD: RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala deleted file mode 100644 index ee0726cb2f00a..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.execution.datasources - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.types.StructType - -/** - * An optional mix-in for columnar [[FileFormat]]s. This trait provides some helpful metadata when - * debugging a physical query plan. - */ -private[sql] trait ColumnarFileFormat { - _: FileFormat => - - /** Returns the number of columns in this file format required to satisfy the given schema. */ - def columnCountForSchema(sparkSession: SparkSession, readSchema: StructType): Int -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 6610795cfdd21..8f698699b35a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -55,7 +55,6 @@ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} class ParquetFileFormat extends FileFormat - with ColumnarFileFormat with DataSourceRegister with Logging with Serializable { @@ -73,14 +72,6 @@ class ParquetFileFormat override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] - override def columnCountForSchema(sparkSession: SparkSession, readSchema: StructType): Int = { - val converter = new SparkToParquetSchemaConverter( - sparkSession.sessionState.conf.writeLegacyParquetFormat, - sparkSession.sessionState.conf.parquetOutputTimestampType) - val parquetSchema = converter.convert(readSchema) - parquetSchema.getPaths.size - } - override def prepareWrite( sparkSession: SparkSession, job: Job, From e213471d9538e7368cfd02567f3da8f67c6878d6 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 11 Jun 2018 19:34:19 -0700 Subject: [PATCH 08/24] Add test coverage for the enhancements to "is not null" constraint inference for complex type extractors --- .../InferFiltersFromConstraintsSuite.scala | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index e4671f0d1cce6..0f0865ffd5e83 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ class InferFiltersFromConstraintsSuite extends PlanTest { @@ -41,6 +42,11 @@ class InferFiltersFromConstraintsSuite extends PlanTest { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + // Separate relation for testing complex type extractors + val testComplexTypeRelation = LocalRelation('a.struct('b.int), 'c.array(IntegerType), + 'd.array(StructType(Array(StructField("e", IntegerType)))), + 'f.map(StringType, IntegerType)) + private def testConstraintsAfterJoin( x: LogicalPlan, y: LogicalPlan, @@ -62,6 +68,37 @@ class InferFiltersFromConstraintsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("filter: filter out nulls on struct field in condition") { + val originalQuery = testComplexTypeRelation.where($"a.b" === 1).analyze + val correctAnswer = testComplexTypeRelation.where(IsNotNull($"a.b") && $"a.b" === 1).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("filter: filter out nulls on array item in condition") { + val originalQuery = testComplexTypeRelation.where('c.getItem(0) === 1).analyze + val correctAnswer = testComplexTypeRelation + .where(IsNotNull('c.getItem(0)) &&'c.getItem(0) === 1).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("filter: filter out nulls on array struct field in condition") { + val originalQuery = testComplexTypeRelation.where($"d.e" === Literal(Array(1))).analyze + val correctAnswer = testComplexTypeRelation + .where(IsNotNull($"d.e") && $"d.e" === Literal(Array(1))).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("filter: filter out nulls on map value in condition") { + val originalQuery = testComplexTypeRelation.where(GetMapValue('f, "key") === 1).analyze + val correctAnswer = testComplexTypeRelation + .where(IsNotNull(GetMapValue('f, "key")) && GetMapValue('f, "key") === 1).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + test("single inner join: filter out values on either side on equi-join keys") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) From 9e6ef5f42705efa20a17d911e26e66f372e21120 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sun, 24 Jun 2018 12:49:33 -0700 Subject: [PATCH 09/24] Revert changes to QueryPlanConstraints.scala and basicPhysicalOperators.scala. Remove related unit tests. This change removes support for schema pruning on filters involving attributes of complex type --- .../plans/logical/QueryPlanConstraints.scala | 22 +++++------ .../InferFiltersFromConstraintsSuite.scala | 37 ------------------- .../execution/basicPhysicalOperators.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 15 -------- .../parquet/ParquetSchemaPruningSuite.scala | 10 ----- 5 files changed, 11 insertions(+), 77 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index 5bfb2ec4c678b..cc352c59dff80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -81,9 +81,8 @@ trait ConstraintHelper { /** * Infers a set of `isNotNull` constraints from null intolerant expressions as well as - * non-nullable attributes and complex type extractors. For example, if an expression is of the - * form (`a > 5`), this returns a constraint of the form `isNotNull(a)`. For an expression of the - * form (`a.b > 5`), this returns the more precise constraint `isNotNull(a.b)`. + * non-nullable attributes. For e.g., if an expression is of the form (`a > 5`), this + * returns a constraint of the form `isNotNull(a)` */ def constructIsNotNullConstraints( constraints: Set[Expression], @@ -100,28 +99,27 @@ trait ConstraintHelper { } /** - * Infer the Attribute and ExtractValue-specific IsNotNull constraints from the null intolerant - * child expressions of constraints. + * Infer the Attribute-specific IsNotNull constraints from the null intolerant child expressions + * of constraints. */ private def inferIsNotNullConstraints(constraint: Expression): Seq[Expression] = constraint match { // When the root is IsNotNull, we can push IsNotNull through the child null intolerant // expressions - case IsNotNull(expr) => scanNullIntolerantField(expr).map(IsNotNull(_)) + case IsNotNull(expr) => scanNullIntolerantAttribute(expr).map(IsNotNull(_)) // Constraints always return true for all the inputs. That means, null will never be returned. // Thus, we can infer `IsNotNull(constraint)`, and also push IsNotNull through the child // null intolerant expressions. - case _ => scanNullIntolerantField(constraint).map(IsNotNull(_)) + case _ => scanNullIntolerantAttribute(constraint).map(IsNotNull(_)) } /** - * Recursively explores the expressions which are null intolerant and returns all attributes and - * complex type extractors in these expressions. + * Recursively explores the expressions which are null intolerant and returns all attributes + * in these expressions. */ - private def scanNullIntolerantField(expr: Expression): Seq[Expression] = expr match { - case ev: ExtractValue => Seq(ev) + private def scanNullIntolerantAttribute(expr: Expression): Seq[Attribute] = expr match { case a: Attribute => Seq(a) - case _: NullIntolerant => expr.children.flatMap(scanNullIntolerantField) + case _: NullIntolerant => expr.children.flatMap(scanNullIntolerantAttribute) case _ => Seq.empty[Attribute] } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index 0f0865ffd5e83..e4671f0d1cce6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ class InferFiltersFromConstraintsSuite extends PlanTest { @@ -42,11 +41,6 @@ class InferFiltersFromConstraintsSuite extends PlanTest { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - // Separate relation for testing complex type extractors - val testComplexTypeRelation = LocalRelation('a.struct('b.int), 'c.array(IntegerType), - 'd.array(StructType(Array(StructField("e", IntegerType)))), - 'f.map(StringType, IntegerType)) - private def testConstraintsAfterJoin( x: LogicalPlan, y: LogicalPlan, @@ -68,37 +62,6 @@ class InferFiltersFromConstraintsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("filter: filter out nulls on struct field in condition") { - val originalQuery = testComplexTypeRelation.where($"a.b" === 1).analyze - val correctAnswer = testComplexTypeRelation.where(IsNotNull($"a.b") && $"a.b" === 1).analyze - val optimized = Optimize.execute(originalQuery) - comparePlans(optimized, correctAnswer) - } - - test("filter: filter out nulls on array item in condition") { - val originalQuery = testComplexTypeRelation.where('c.getItem(0) === 1).analyze - val correctAnswer = testComplexTypeRelation - .where(IsNotNull('c.getItem(0)) &&'c.getItem(0) === 1).analyze - val optimized = Optimize.execute(originalQuery) - comparePlans(optimized, correctAnswer) - } - - test("filter: filter out nulls on array struct field in condition") { - val originalQuery = testComplexTypeRelation.where($"d.e" === Literal(Array(1))).analyze - val correctAnswer = testComplexTypeRelation - .where(IsNotNull($"d.e") && $"d.e" === Literal(Array(1))).analyze - val optimized = Optimize.execute(originalQuery) - comparePlans(optimized, correctAnswer) - } - - test("filter: filter out nulls on map value in condition") { - val originalQuery = testComplexTypeRelation.where(GetMapValue('f, "key") === 1).analyze - val correctAnswer = testComplexTypeRelation - .where(IsNotNull(GetMapValue('f, "key")) && GetMapValue('f, "key") === 1).analyze - val optimized = Optimize.execute(originalQuery) - comparePlans(optimized, correctAnswer) - } - test("single inner join: filter out values on either side on equi-join keys") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index c29c336a974f5..9434ceb7cd16c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -162,9 +162,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) val generatedIsNotNullChecks = new Array[Boolean](notNullPreds.length) val generated = otherPreds.map { c => val nullChecks = c.references.map { r => - val idx = notNullPreds.indexWhere { n => - n.asInstanceOf[IsNotNull].child.references.contains(r) - } + val idx = notNullPreds.indexWhere { n => n.asInstanceOf[IsNotNull].child.semanticEquals(r)} if (idx != -1 && !generatedIsNotNullChecks(idx)) { generatedIsNotNullChecks(idx) = true // Use the child's output. The nullability is what the child produced. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 430101b16459c..b0e22a51e7611 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -2499,21 +2499,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df, df.collect()) } - test("SPARK-4502: Nested column pruning shouldn't fail filter") { - withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { - withTempPath { dir => - val path = dir.getCanonicalPath - val data = - """{"a":{"b":1,"c":2}} - |{}""".stripMargin - Seq(data).toDF().repartition(1).write.text(path) - checkAnswer( - spark.read.json(path).filter($"a.b" > 1).select($"a.b"), - Seq.empty) - } - } - } - test("SPARK-24313: access map with binary keys") { val mapWithBinaryKey = map(lit(Array[Byte](1.toByte)), lit(1)) checkAnswer(spark.range(1).select(mapWithBinaryKey.getItem(Array[Byte](1.toByte))), Row(1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index b7f4858f4912c..943e7bab231f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -76,16 +76,6 @@ class ParquetSchemaPruningSuite Row(null, "6242 Ash Street") :: Nil) } - testSchemaPruning("partial schema intersection - filter on subfield") { - val query = - sql("select name.middle, name.first, pets, address from contacts where " + - "name.first = 'Janet' and p=2") - checkScanSchemata(query, - "struct,pets:int,address:string>") - checkAnswer(query, - Row(null, "Janet", null, "567 Maple Drive") :: Nil) - } - testSchemaPruning("no unnecessary schema pruning") { val query = sql("select name.last, name.middle, name.first, relatives[''].last, " + From e6ea9c22bd788cce2b3d828e5b56c0b73662ff54 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sun, 24 Jun 2018 12:55:49 -0700 Subject: [PATCH 10/24] Revert a whitespace change in DataSourceScanExec.scala --- .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 40b8ebed607f2..36ed016773b67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -303,6 +303,7 @@ case class FileSourceScanExec( } getOrElse { withOptPartitionCount } + withSelectedBucketsCount } From 2d02ab30a9a28641490fb7d555bf32259cf355af Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sat, 21 Jul 2018 02:41:11 -0700 Subject: [PATCH 11/24] Remove modifications to ParquetFileFormat.scala and ParquetReadSupport.scala, marking broken tests in ParquetSchemaPruningSuite.scala as "ignored" --- .../parquet/ParquetFileFormat.scala | 5 +- .../parquet/ParquetReadSupport.scala | 57 +++---------------- .../parquet/ParquetSchemaPruningSuite.scala | 14 ++--- 3 files changed, 16 insertions(+), 60 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 8f698699b35a2..b2409f3470e73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -423,12 +423,11 @@ class ParquetFileFormat } else { logDebug(s"Falling back to parquet-mr") // ParquetRecordReader returns UnsafeRow - val readSupport = new ParquetReadSupport(convertTz, true) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](readSupport) + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 00db3cc62ff4f..40ce5d5e0564e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -47,25 +47,16 @@ import org.apache.spark.sql.types._ * * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. - * - * @param parquetMrCompatibility support reading with parquet-mr or Spark's built-in Parquet reader */ -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], - parquetMrCompatibility: Boolean) +private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) extends ReadSupport[UnsafeRow] with Logging { private var catalystRequestedSchema: StructType = _ - /** - * Construct a [[ParquetReadSupport]] with [[convertTz]] set to [[None]] and - * [[parquetMrCompatibility]] set to [[false]]. - * - * We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only - * used in the vectorized reader, where we get the convertTz value directly, and the value here - * is ignored. Further, we set [[parquetMrCompatibility]] to [[false]] as this constructor is only - * called by the Spark reader. - */ def this() { - this(None, false) + // We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only + // used in the vectorized reader, where we get the convertTz value directly, and the value here + // is ignored. + this(None) } /** @@ -80,22 +71,9 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], StructType.fromString(schemaString) } - val clippedParquetSchema = + val parquetRequestedSchema = ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema) - val parquetRequestedSchema = if (parquetMrCompatibility) { - // Parquet-mr will throw an exception if we try to read a superset of the file's schema. - // Therefore, we intersect our clipped schema with the underlying file's schema - ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema) - .map(intersectionGroup => - new MessageType(intersectionGroup.getName, intersectionGroup.getFields)) - .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE) - } else { - // Spark's built-in Parquet reader will throw an exception in some cases if the requested - // schema is not the same as the clipped schema - clippedParquetSchema - } - new ReadContext(parquetRequestedSchema, Map.empty[String, String].asJava) } @@ -118,7 +96,7 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], |Parquet form: |$parquetRequestedSchema |Catalyst form: - |${catalystRequestedSchema.prettyJson} + |$catalystRequestedSchema """.stripMargin } @@ -310,27 +288,6 @@ private[parquet] object ParquetReadSupport { } } - /** - * Computes the structural intersection between two Parquet group types. - */ - private def intersectParquetGroups( - groupType1: GroupType, groupType2: GroupType): Option[GroupType] = { - val fields = - groupType1.getFields.asScala - .filter(field => groupType2.containsField(field.getName)) - .flatMap { - case field1: GroupType => - intersectParquetGroups(field1, groupType2.getType(field1.getName).asGroupType) - case field1 => Some(field1) - } - - if (fields.nonEmpty) { - Some(groupType1.withNewFields(fields.asJava)) - } else { - None - } - } - def expandUDT(schema: StructType): StructType = { def expand(dataType: DataType): DataType = { dataType match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 943e7bab231f1..b05b2b7cc843c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -68,7 +68,7 @@ class ParquetSchemaPruningSuite checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - testSchemaPruning("partial schema intersection - select missing subfield") { + ignore("partial schema intersection - select missing subfield") { val query = sql("select name.middle, address from contacts where p=2") checkScanSchemata(query, "struct,address:string>") checkAnswer(query, @@ -102,20 +102,20 @@ class ParquetSchemaPruningSuite } private def testSchemaPruning(testName: String)(testThunk: => Unit) { - withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") { - test(s"Standard mode - without partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + test(s"Spark vectorized reader - without partition data column - $testName") { withContacts(testThunk) } - test(s"Standard mode - with partition data column - $testName") { + test(s"Spark vectorized reader - with partition data column - $testName") { withContactsWithDataPartitionColumn(testThunk) } } - withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "true") { - test(s"Legacy mode - without partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + test(s"Parquet-mr reader - without partition data column - $testName") { withContacts(testThunk) } - test(s"Legacy mode - with partition data column - $testName") { + test(s"Parquet-mr reader - with partition data column - $testName") { withContactsWithDataPartitionColumn(testThunk) } } From cfffc95df4fa54510aa8b1e06ace141752516ebd Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sat, 21 Jul 2018 14:47:45 -0700 Subject: [PATCH 12/24] PR review: simplify some syntax and add a code doc --- .../sql/catalyst/planning/ProjectionOverSchema.scala | 10 ++-------- .../sql/catalyst/planning/SelectedFieldSuite.scala | 3 ++- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala index e305676ffa8a1..39d8a102d6050 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala @@ -36,10 +36,7 @@ case class ProjectionOverSchema(schema: StructType) { case a @ AttributeReference(name, _, _, _) if (fieldNames.contains(name)) => Some(a.copy(dataType = schema(name).dataType)(a.exprId, a.qualifier)) case GetArrayItem(child, arrayItemOrdinal) => - getProjection(child).map { - case projection => - GetArrayItem(projection, arrayItemOrdinal) - } + getProjection(child).map { projection => GetArrayItem(projection, arrayItemOrdinal) } case GetArrayStructFields(child, StructField(name, _, _, _), _, numFields, containsNull) => getProjection(child).map(p => (p, p.dataType)).map { case (projection, ArrayType(projSchema @ StructType(_), _)) => @@ -47,10 +44,7 @@ case class ProjectionOverSchema(schema: StructType) { projSchema(name), projSchema.fieldIndex(name), projSchema.size, containsNull) } case GetMapValue(child, key) => - getProjection(child).map { - case projection => - GetMapValue(projection, key) - } + getProjection(child).map { projection => GetMapValue(projection, key) } case GetStructFieldObject(child, StructField(name, _, _, _)) => getProjection(child).map(p => (p, p.dataType)).map { case (projection, projSchema @ StructType(_)) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala index ce1bda5301f78..f4ed7570ab258 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala @@ -352,7 +352,7 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { // defined by the given field private def testSelect(selectExpr: String, otherSelectExprs: String*)(expected: StructField) { val selectExprs = selectExpr +: otherSelectExprs - test(s"SELECT " + selectExprs.map(s => s""""$s"""").mkString(", ") + " should select the schema\n" + + test(s"SELECT ${selectExprs.map(s => s""""$s"""").mkString(", ")} should select the schema\n" + indent(StructType(expected :: Nil).treeString)) { for (selectExpr <- selectExprs) { assertSelect(selectExpr, expected) @@ -383,5 +383,6 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { SelectedField.unapply(analyzed.expressions.head) } + // Indent every line in `string` by four spaces private def indent(string: String) = string.replaceAll("(?m)^", " ") } From 2779351386454f2de4f80100ae8e146357f4bee1 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Fri, 27 Jul 2018 23:40:38 -0700 Subject: [PATCH 13/24] When creating a pruned schema by merging an array of root structs, sort the fields of the merged schema according to their order in the original schema. The underlying merge function we use, StructType.merge, does not necessarily merge the fields of two structs into the order which we require --- .../parquet/ParquetSchemaPruning.scala | 41 ++++++- .../parquet/ParquetSchemaPruningSuite.scala | 101 +++++++++++++----- 2 files changed, 114 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index f31d24a549223..47465051a7394 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -51,12 +51,19 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // If [[requestedRootFields]] includes a nested field, continue. Otherwise, // return [[op]] if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) { - val prunedSchema = requestedRootFields + // 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 RootField(field, _) => StructType(Array(field)) } .reduceLeft(_ merge _) val dataSchemaFieldNames = dataSchema.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]] val prunedDataSchema = - StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) + sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType] // If the data schema is different from the pruned data schema, continue. Otherwise, // return [[op]]. We effect this comparison by counting the number of "leaf" fields in @@ -145,6 +152,36 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { } } + /** + * 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. + */ + private def sortLeftFieldsByRight(left: DataType, right: DataType): DataType = + (left, right) match { + case (ArrayType(leftElementType, containsNull), ArrayType(rightElementType, _)) => + ArrayType( + sortLeftFieldsByRight(leftElementType, rightElementType), + containsNull) + case (MapType(leftKeyType, leftValueType, containsNull), + MapType(rightKeyType, rightValueType, _)) => + MapType( + sortLeftFieldsByRight(leftKeyType, rightKeyType), + sortLeftFieldsByRight(leftValueType, rightValueType), + containsNull) + case (leftStruct: StructType, rightStruct: StructType) => + val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains) + val sortedLeftFields = filteredRightFieldNames.map { fieldName => + val leftFieldType = leftStruct(fieldName).dataType + val rightFieldType = rightStruct(fieldName).dataType + val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType) + StructField(fieldName, sortedLeftFieldType) + } + StructType(sortedLeftFields) + case (left, _) => left + } + /** * A "root" schema field (aka top-level, no-parent) and whether it was derived from * an attribute or had a proper child. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index b05b2b7cc843c..7f47d7b133de9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -30,47 +30,96 @@ class ParquetSchemaPruningSuite with FileSchemaPruningTest with SharedSQLContext { case class FullName(first: String, middle: String, last: String) - case class Contact(name: FullName, address: String, pets: Int, friends: Array[FullName] = Array(), + case class Contact( + id: Int, + name: FullName, + address: String, + pets: Int, + friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map()) + val janeDoe = FullName("Jane", "X.", "Doe") + val johnDoe = FullName("John", "Y.", "Doe") + val susanSmith = FullName("Susan", "Z.", "Smith") + val contacts = - Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) :: - Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil + Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), + relatives = Map("brother" -> johnDoe)) :: + Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe)) :: Nil case class Name(first: String, last: String) - case class BriefContact(name: Name, address: String) + case class BriefContact(id: Int, name: Name, address: String) val briefContacts = - BriefContact(Name("Janet", "Jones"), "567 Maple Drive") :: - BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil + BriefContact(2, Name("Janet", "Jones"), "567 Maple Drive") :: + BriefContact(3, Name("Jim", "Jones"), "6242 Ash Street") :: Nil - case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int, - friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), p: Int) + case class ContactWithDataPartitionColumn( + id: Int, + name: FullName, + address: String, + pets: Int, + friends: Array[FullName] = Array(), + relatives: Map[String, FullName] = Map(), + p: Int) - case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int) + case class BriefContactWithDataPartitionColumn(id: Int, name: Name, address: String, p: Int) val contactsWithDataPartitionColumn = - contacts.map { case Contact(name, address, pets, friends, relatives) => - ContactWithDataPartitionColumn(name, address, pets, friends, relatives, 1) } + contacts.map { case Contact(id, name, address, pets, friends, relatives) => + ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, 1) } val briefContactsWithDataPartitionColumn = - briefContacts.map { case BriefContact(name: Name, address: String) => - BriefContactWithDataPartitionColumn(name, address, 2) } + briefContacts.map { case BriefContact(id, name, address) => + BriefContactWithDataPartitionColumn(id, name, address, 2) } testSchemaPruning("select a single complex field") { - val query = sql("select name.middle from contacts") - checkScanSchemata(query, "struct>") + val query = sql("select name.middle from contacts order by id") + checkScanSchemata(query, "struct>") checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) } + testSchemaPruning("select a single complex field and its parent struct") { + val query = sql("select name.middle, name from contacts order by id") + checkScanSchemata(query, "struct>") + checkAnswer(query, + Row("X.", Row("Jane", "X.", "Doe")) :: + Row("Y.", Row("John", "Y.", "Doe")) :: + Row(null, Row("Janet", null, "Jones")) :: + Row(null, Row("Jim", null, "Jones")) :: + Nil) + } + + testSchemaPruning("select a single complex field array and its parent struct array") { + val query = sql("select friends.middle, friends from contacts where p=1 order by id") + checkScanSchemata(query, + "struct>>") + checkAnswer(query, + Row(Array("Z."), Array(Row("Susan", "Z.", "Smith"))) :: + Row(Array.empty[String], Array.empty[Row]) :: + Nil) + } + + testSchemaPruning("select a single complex field from a map entry and its parent map entry") { + val query = + sql("select relatives[\"brother\"].middle, relatives[\"brother\"] from contacts where p=1 " + + "order by id") + checkScanSchemata(query, + "struct>>") + checkAnswer(query, + Row("Y.", Row("John", "Y.", "Doe")) :: + Row(null, null) :: + Nil) + } + testSchemaPruning("select a single complex field and the partition column") { - val query = sql("select name.middle, p from contacts") - checkScanSchemata(query, "struct>") + val query = sql("select name.middle, p from contacts order by id") + checkScanSchemata(query, "struct>") checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } ignore("partial schema intersection - select missing subfield") { - val query = sql("select name.middle, address from contacts where p=2") - checkScanSchemata(query, "struct,address:string>") + val query = sql("select name.middle, address from contacts where p=2 order by id") + checkScanSchemata(query, "struct,address:string>") checkAnswer(query, Row(null, "567 Maple Drive") :: Row(null, "6242 Ash Street") :: Nil) @@ -78,25 +127,25 @@ class ParquetSchemaPruningSuite testSchemaPruning("no unnecessary schema pruning") { val query = - sql("select name.last, name.middle, name.first, relatives[''].last, " + + sql("select id, name.last, name.middle, name.first, relatives[''].last, " + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + - "friends[0].first, pets, address from contacts where p=2") + "friends[0].first, pets, address from contacts where p=2 order by id") // We've selected every field in the schema. Therefore, no schema pruning should be performed. // We check this by asserting that the scanned schema of the query is identical to the schema // of the contacts relation, even though the fields are selected in different orders. checkScanSchemata(query, - "struct,address:string,pets:int," + + "struct,address:string,pets:int," + "friends:array>," + "relatives:map>>") checkAnswer(query, - Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: - Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: + Row(2, "Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: + Row(3, "Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: Nil) } testSchemaPruning("empty schema intersection") { - val query = sql("select name.middle from contacts where p=2") - checkScanSchemata(query, "struct>") + val query = sql("select name.middle from contacts where p=2 order by id") + checkScanSchemata(query, "struct>") checkAnswer(query, Row(null) :: Row(null) :: Nil) } From 9329f774d43bdc6d507084e22445a89fdb4a06cd Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sat, 4 Aug 2018 14:20:57 -0700 Subject: [PATCH 14/24] Re-enable ignored test in ParquetSchemaPruningSuite.scala that is passing now --- .../datasources/parquet/ParquetSchemaPruningSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 7f47d7b133de9..7310a5764846f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -117,7 +117,7 @@ class ParquetSchemaPruningSuite checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - ignore("partial schema intersection - select missing subfield") { + testSchemaPruning("partial schema intersection - select missing subfield") { val query = sql("select name.middle, address from contacts where p=2 order by id") checkScanSchemata(query, "struct,address:string>") checkAnswer(query, From ec313c13e0d2b0018fc3761819e2e27a520e001b Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sat, 4 Aug 2018 14:21:35 -0700 Subject: [PATCH 15/24] Enable schema pruning by default --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/SchemaPruningTest.scala | 52 ------------------- .../sql/execution/FileSchemaPruningTest.scala | 3 +- 3 files changed, 2 insertions(+), 55 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6a3b7df5e0be5..1557e8b98aecb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1392,7 +1392,7 @@ object SQLConf { "reading unnecessary nested column data. Currently Parquet is the only data source that " + "implements this optimization.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala deleted file mode 100644 index 88e30e03d2d3b..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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 - -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED - -/** - * A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled. - * Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true. - */ -private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll { - private var originalConfSchemaPruningEnabled = false - - override protected def beforeAll(): Unit = { - // Call `withSQLConf` eagerly because some subtypes of `PlanTest` (I'm looking at you, - // `SQLTestUtils`) override `withSQLConf` to reset the existing `SQLConf` with a new one without - // copying existing settings first. This here is an awful, ugly way to get around that behavior - // by initializing the "real" `SQLConf` with an noop call to `withSQLConf`. I don't want to risk - // "fixing" the downstream behavior, breaking everything else that's expecting these semantics. - // Oh well... - withSQLConf()(()) - originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled - conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true) - super.beforeAll() - } - - override protected def afterAll(): Unit = { - try { - super.afterAll() - } finally { - conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala index e7b634d227c85..15d103d7f90f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala @@ -21,11 +21,10 @@ import org.scalactic.Equality import org.scalatest.Assertions import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.StructType -private[sql] trait FileSchemaPruningTest extends SchemaPruningTest { +private[sql] trait FileSchemaPruningTest { _: Assertions => private val schemaEquality = new Equality[StructType] { From 42aff39f4591711bbc4ec4b6cfe8b94751223d54 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sun, 5 Aug 2018 00:21:48 -0700 Subject: [PATCH 16/24] Revert "Enable schema pruning by default" This reverts commit a87b589c70af69d25424b03b52e29165ab4e0b5f. --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/SchemaPruningTest.scala | 52 +++++++++++++++++++ .../sql/execution/FileSchemaPruningTest.scala | 3 +- 3 files changed, 55 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1557e8b98aecb..6a3b7df5e0be5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1392,7 +1392,7 @@ object SQLConf { "reading unnecessary nested column data. Currently Parquet is the only data source that " + "implements this optimization.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala new file mode 100644 index 0000000000000..88e30e03d2d3b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED + +/** + * A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled. + * Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true. + */ +private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll { + private var originalConfSchemaPruningEnabled = false + + override protected def beforeAll(): Unit = { + // Call `withSQLConf` eagerly because some subtypes of `PlanTest` (I'm looking at you, + // `SQLTestUtils`) override `withSQLConf` to reset the existing `SQLConf` with a new one without + // copying existing settings first. This here is an awful, ugly way to get around that behavior + // by initializing the "real" `SQLConf` with an noop call to `withSQLConf`. I don't want to risk + // "fixing" the downstream behavior, breaking everything else that's expecting these semantics. + // Oh well... + withSQLConf()(()) + originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true) + super.beforeAll() + } + + override protected def afterAll(): Unit = { + try { + super.afterAll() + } finally { + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala index 15d103d7f90f9..e7b634d227c85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala @@ -21,10 +21,11 @@ import org.scalactic.Equality import org.scalatest.Assertions import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.StructType -private[sql] trait FileSchemaPruningTest { +private[sql] trait FileSchemaPruningTest extends SchemaPruningTest { _: Assertions => private val schemaEquality = new Equality[StructType] { From 71f4c7b87822a485a82da3dbed918b6670f85b8b Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Sun, 5 Aug 2018 00:42:14 -0700 Subject: [PATCH 17/24] Add a method to not only check a query's scan schemata, but verify that it can actually return results. Then remove the ORDER BY clauses of the test queries, check those queries, and check the answers of those queries with ordering by id. Phew! --- .../sql/execution/FileSchemaPruningTest.scala | 9 ++- .../parquet/ParquetSchemaPruningSuite.scala | 63 +++++++++---------- 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala index e7b634d227c85..f74b801707216 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala @@ -36,7 +36,14 @@ private[sql] trait FileSchemaPruningTest extends SchemaPruningTest { } } - protected def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + protected def checkScan(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + checkScanSchemata(df, expectedSchemaCatalogStrings: _*) + // We check here that we can execute the query without throwing an exception. The results + // themselves are irrelevant, and should be checked elsewhere as needed + df.collect() + } + + private def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { val fileSourceScanSchemata = df.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan.requiredSchema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 7310a5764846f..d9e4696c3fa15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -73,15 +73,15 @@ class ParquetSchemaPruningSuite BriefContactWithDataPartitionColumn(id, name, address, 2) } testSchemaPruning("select a single complex field") { - val query = sql("select name.middle from contacts order by id") - checkScanSchemata(query, "struct>") - checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) + val query = sql("select name.middle from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) } testSchemaPruning("select a single complex field and its parent struct") { - val query = sql("select name.middle, name from contacts order by id") - checkScanSchemata(query, "struct>") - checkAnswer(query, + val query = sql("select name.middle, name from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row("X.", Row("Jane", "X.", "Doe")) :: Row("Y.", Row("John", "Y.", "Doe")) :: Row(null, Row("Janet", null, "Jones")) :: @@ -90,10 +90,10 @@ class ParquetSchemaPruningSuite } testSchemaPruning("select a single complex field array and its parent struct array") { - val query = sql("select friends.middle, friends from contacts where p=1 order by id") - checkScanSchemata(query, - "struct>>") - checkAnswer(query, + val query = sql("select friends.middle, friends from contacts where p=1") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), Row(Array("Z."), Array(Row("Susan", "Z.", "Smith"))) :: Row(Array.empty[String], Array.empty[Row]) :: Nil) @@ -101,26 +101,26 @@ class ParquetSchemaPruningSuite testSchemaPruning("select a single complex field from a map entry and its parent map entry") { val query = - sql("select relatives[\"brother\"].middle, relatives[\"brother\"] from contacts where p=1 " + - "order by id") - checkScanSchemata(query, - "struct>>") - checkAnswer(query, + sql("select relatives[\"brother\"].middle, relatives[\"brother\"] from contacts where p=1") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), Row("Y.", Row("John", "Y.", "Doe")) :: Row(null, null) :: Nil) } testSchemaPruning("select a single complex field and the partition column") { - val query = sql("select name.middle, p from contacts order by id") - checkScanSchemata(query, "struct>") - checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) + val query = sql("select name.middle, p from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - testSchemaPruning("partial schema intersection - select missing subfield") { - val query = sql("select name.middle, address from contacts where p=2 order by id") - checkScanSchemata(query, "struct,address:string>") - checkAnswer(query, + ignore("partial schema intersection - select missing subfield") { + val query = sql("select name.middle, address from contacts where p=2") + checkScan(query, "struct,address:string>") + checkAnswer(query.orderBy("id"), Row(null, "567 Maple Drive") :: Row(null, "6242 Ash Street") :: Nil) } @@ -129,24 +129,24 @@ class ParquetSchemaPruningSuite val query = sql("select id, name.last, name.middle, name.first, relatives[''].last, " + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + - "friends[0].first, pets, address from contacts where p=2 order by id") + "friends[0].first, pets, address from contacts where p=2") // We've selected every field in the schema. Therefore, no schema pruning should be performed. // We check this by asserting that the scanned schema of the query is identical to the schema // of the contacts relation, even though the fields are selected in different orders. - checkScanSchemata(query, + checkScan(query, "struct,address:string,pets:int," + "friends:array>," + "relatives:map>>") - checkAnswer(query, + checkAnswer(query.orderBy("id"), Row(2, "Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: Row(3, "Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: Nil) } testSchemaPruning("empty schema intersection") { - val query = sql("select name.middle from contacts where p=2 order by id") - checkScanSchemata(query, "struct>") - checkAnswer(query, + val query = sql("select name.middle from contacts where p=2") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row(null) :: Row(null) :: Nil) } @@ -170,13 +170,6 @@ class ParquetSchemaPruningSuite } } - private def withContactTables(testThunk: => Unit) { - info("testing table without partition data column") - withContacts(testThunk) - info("testing table with partition data column") - withContactsWithDataPartitionColumn(testThunk) - } - private def withContacts(testThunk: => Unit) { withTempPath { dir => val path = dir.getCanonicalPath From 0e5594b6ac1dcb94f3f0166e66a7d4e7eae3d00c Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Thu, 9 Aug 2018 00:17:31 -0700 Subject: [PATCH 18/24] Revert "Revert "Enable schema pruning by default"" This reverts commit 323875a091961660a967c42615b7491e9d752cfd. --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/SchemaPruningTest.scala | 52 ------------------- .../sql/execution/FileSchemaPruningTest.scala | 3 +- 3 files changed, 2 insertions(+), 55 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6a3b7df5e0be5..1557e8b98aecb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1392,7 +1392,7 @@ object SQLConf { "reading unnecessary nested column data. Currently Parquet is the only data source that " + "implements this optimization.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala deleted file mode 100644 index 88e30e03d2d3b..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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 - -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED - -/** - * A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled. - * Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true. - */ -private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll { - private var originalConfSchemaPruningEnabled = false - - override protected def beforeAll(): Unit = { - // Call `withSQLConf` eagerly because some subtypes of `PlanTest` (I'm looking at you, - // `SQLTestUtils`) override `withSQLConf` to reset the existing `SQLConf` with a new one without - // copying existing settings first. This here is an awful, ugly way to get around that behavior - // by initializing the "real" `SQLConf` with an noop call to `withSQLConf`. I don't want to risk - // "fixing" the downstream behavior, breaking everything else that's expecting these semantics. - // Oh well... - withSQLConf()(()) - originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled - conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true) - super.beforeAll() - } - - override protected def afterAll(): Unit = { - try { - super.afterAll() - } finally { - conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala index f74b801707216..52911034ff069 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala @@ -21,11 +21,10 @@ import org.scalactic.Equality import org.scalatest.Assertions import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.StructType -private[sql] trait FileSchemaPruningTest extends SchemaPruningTest { +private[sql] trait FileSchemaPruningTest { _: Assertions => private val schemaEquality = new Equality[StructType] { From 1573ae888d651a51e0d60683117714fba7c55fb0 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Fri, 17 Aug 2018 16:46:46 -0700 Subject: [PATCH 19/24] Revert changes to ParquetTest.scala. I'm sure they were useful at some point in the past, but now they're unused --- .../datasources/parquet/ParquetTest.scala | 48 +++++-------------- 1 file changed, 11 insertions(+), 37 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index bf15525cbda34..f05f5722af51a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -59,68 +59,42 @@ private[sql] trait ParquetTest extends SQLTestUtils { } /** - * Writes `df` to a Parquet file, which is then passed to `f` and will be deleted after `f` + * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` * returns. */ - protected def asParquetFile - (df: DataFrame) + protected def withParquetFile[T <: Product: ClassTag: TypeTag] + (data: Seq[T]) (f: String => Unit): Unit = { withTempPath { file => - df.write.parquet(file.getCanonicalPath) + spark.createDataFrame(data).write.parquet(file.getCanonicalPath) f(file.getCanonicalPath) } } - /** - * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` - * returns. - */ - protected def withParquetFile[T <: Product: ClassTag: TypeTag] - (data: Seq[T]) - (f: String => Unit): Unit = asParquetFile(spark.createDataFrame(data))(f) - - /** - * Writes `df` to a Parquet file and reads it back as a [[DataFrame]], - * which is then passed to `f`. The Parquet file will be deleted after `f` returns. - */ - protected def asParquetDataFrame - (df: DataFrame, testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = - asParquetFile(df)(path => readParquetFile(path.toString, testVectorized)(f)) - /** * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] (data: Seq[T], testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = - asParquetDataFrame(spark.createDataFrame(data), testVectorized)(f) + (f: DataFrame => Unit): Unit = { + withParquetFile(data)(path => readParquetFile(path.toString, testVectorized)(f)) + } /** - * Writes `df` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a + * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a * temporary table named `tableName`, then call `f`. The temporary table together with the * Parquet file will be dropped/deleted after `f` returns. */ - protected def asParquetTable - (df: DataFrame, tableName: String, testVectorized: Boolean = true) + protected def withParquetTable[T <: Product: ClassTag: TypeTag] + (data: Seq[T], tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { - asParquetDataFrame(df, testVectorized) { df => + withParquetDataFrame(data, testVectorized) { df => df.createOrReplaceTempView(tableName) withTempView(tableName)(f) } } - /** - * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a - * temporary table named `tableName`, then call `f`. The temporary table together with the - * Parquet file will be dropped/deleted after `f` returns. - */ - protected def withParquetTable[T <: Product: ClassTag: TypeTag] - (data: Seq[T], tableName: String, testVectorized: Boolean = true) - (f: => Unit): Unit = - asParquetTable(spark.createDataFrame(data), tableName, testVectorized)(f) - protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( data: Seq[T], path: File): Unit = { spark.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) From 09dd655b854579fd7d3abb8acb942d13e873d62d Mon Sep 17 00:00:00 2001 From: Adam Jacques Date: Thu, 2 Aug 2018 10:18:33 -0700 Subject: [PATCH 20/24] Refactor code based on code review comments Co-author: Hyukjin Kwon --- .../sql/execution}/GetStructFieldObject.scala | 4 +- .../sql/execution}/ProjectionOverSchema.scala | 23 +- .../spark/sql/execution}/SelectedField.scala | 50 +- .../parquet/ParquetSchemaPruning.scala | 189 ++++--- .../sql/execution/FileSchemaPruningTest.scala | 60 --- .../sql/execution}/SelectedFieldSuite.scala | 483 ++++++++++-------- .../parquet/ParquetSchemaPruningSuite.scala | 56 +- 7 files changed, 474 insertions(+), 391 deletions(-) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning => core/src/main/scala/org/apache/spark/sql/execution}/GetStructFieldObject.scala (93%) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning => core/src/main/scala/org/apache/spark/sql/execution}/ProjectionOverSchema.scala (72%) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning => core/src/main/scala/org/apache/spark/sql/execution}/SelectedField.scala (85%) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala rename sql/{catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning => core/src/test/scala/org/apache/spark/sql/execution}/SelectedFieldSuite.scala (61%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala similarity index 93% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala index 033792a9ac728..c88b2f8c034fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/GetStructFieldObject.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.planning +package org.apache.spark.sql.execution import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField} import org.apache.spark.sql.types.StructField @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.StructField * This is in contrast to the [[GetStructField]] case class extractor which returns the field * ordinal instead of the field itself. */ -private[planning] object GetStructFieldObject { +private[execution] object GetStructFieldObject { def unapply(getStructField: GetStructField): Option[(Expression, StructField)] = Some(( getStructField.child, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala similarity index 72% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala index 39d8a102d6050..2236f18b0da12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.planning +package org.apache.spark.sql.execution import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -26,29 +26,32 @@ import org.apache.spark.sql.types._ * are adjusted to fit the schema. All other expressions are left as-is. This * class is motivated by columnar nested schema pruning. */ -case class ProjectionOverSchema(schema: StructType) { +private[execution] case class ProjectionOverSchema(schema: StructType) { private val fieldNames = schema.fieldNames.toSet def unapply(expr: Expression): Option[Expression] = getProjection(expr) private def getProjection(expr: Expression): Option[Expression] = expr match { - case a @ AttributeReference(name, _, _, _) if (fieldNames.contains(name)) => - Some(a.copy(dataType = schema(name).dataType)(a.exprId, a.qualifier)) + case a: AttributeReference if fieldNames.contains(a.name) => + Some(a.copy(dataType = schema(a.name).dataType)(a.exprId, a.qualifier)) case GetArrayItem(child, arrayItemOrdinal) => getProjection(child).map { projection => GetArrayItem(projection, arrayItemOrdinal) } - case GetArrayStructFields(child, StructField(name, _, _, _), _, numFields, containsNull) => - getProjection(child).map(p => (p, p.dataType)).map { + case a: GetArrayStructFields => + getProjection(a.child).map(p => (p, p.dataType)).map { case (projection, ArrayType(projSchema @ StructType(_), _)) => GetArrayStructFields(projection, - projSchema(name), projSchema.fieldIndex(name), projSchema.size, containsNull) + projSchema(a.field.name), + projSchema.fieldIndex(a.field.name), + projSchema.size, + a.containsNull) } case GetMapValue(child, key) => getProjection(child).map { projection => GetMapValue(projection, key) } - case GetStructFieldObject(child, StructField(name, _, _, _)) => + case GetStructFieldObject(child, field: StructField) => getProjection(child).map(p => (p, p.dataType)).map { - case (projection, projSchema @ StructType(_)) => - GetStructField(projection, projSchema.fieldIndex(name)) + case (projection, projSchema: StructType) => + GetStructField(projection, projSchema.fieldIndex(field.name)) } case _ => None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala similarity index 85% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala index dc1e00290bedb..0e7c593f9fb67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.planning +package org.apache.spark.sql.execution import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -24,27 +24,27 @@ import org.apache.spark.sql.types._ * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst * complex type extractor. For example, consider a relation with the following schema: * - * {{{ - * root - * |-- name: struct (nullable = true) - * | |-- first: string (nullable = true) - * | |-- last: string (nullable = true) - * }}} + * {{{ + * root + * |-- name: struct (nullable = true) + * | |-- first: string (nullable = true) + * | |-- last: string (nullable = true) + * }}} * * Further, suppose we take the select expression `name.first`. This will parse into an * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern: * - * {{{ - * GetStructFieldObject( - * AttributeReference("name", StructType(_), _, _), - * StructField("first", StringType, _, _)) - * }}} + * {{{ + * GetStructFieldObject( + * AttributeReference("name", StructType(_), _, _), + * StructField("first", StringType, _, _)) + * }}} * * [[SelectedField]] converts that expression into * - * {{{ - * StructField("name", StructType(Array(StructField("first", StringType)))) - * }}} + * {{{ + * StructField("name", StructType(Array(StructField("first", StringType)))) + * }}} * * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the * same name as its child (or "parent" going right to left in the select expression) and a data @@ -54,7 +54,7 @@ import org.apache.spark.sql.types._ * * @param expr the top-level complex type extractor */ -object SelectedField { +private[execution] object SelectedField { def unapply(expr: Expression): Option[StructField] = { // If this expression is an alias, work on its child instead val unaliased = expr match { @@ -85,16 +85,16 @@ object SelectedField { field @ StructField(name, dataType, nullable, metadata), _, _, _) => val childField = fieldOpt.map(field => StructField(name, wrapStructType(dataType, field), - nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) + nullable, metadata)).orElse(Some(field)) + selectField(child, childField) // Handles case "expr0.field", where "expr0" is of array type. case GetArrayStructFields(child, - field @ StructField(name, dataType, nullable, metadata), _, _, containsNull) => + field @ StructField(name, dataType, nullable, metadata), _, _, _) => val childField = fieldOpt.map(field => StructField(name, wrapStructType(dataType, field), - nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) + nullable, metadata)).orElse(Some(field)) + selectField(child, childField) // Handles case "expr0.field[key]", where "expr0" is of struct type and "expr0.field" is of // map type. case GetMapValue(x @ GetStructFieldObject(child, field @ StructField(name, @@ -102,8 +102,8 @@ object SelectedField { nullable, metadata)), _) => val childField = fieldOpt.map(field => StructField(name, wrapStructType(dataType, field), - nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) + nullable, metadata)).orElse(Some(field)) + selectField(child, childField) // Handles case "expr0.field[key]", where "expr0.field" is of map type. case GetMapValue(child, _) => selectField(child, fieldOpt) @@ -112,8 +112,8 @@ object SelectedField { field @ StructField(name, dataType, nullable, metadata)) => val childField = fieldOpt.map(field => StructField(name, wrapStructType(dataType, field), - nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) + nullable, metadata)).orElse(Some(field)) + selectField(child, childField) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index 47465051a7394..15b8615ec365a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression} -import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ProjectionOverSchema, SelectedField} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} @@ -42,78 +43,28 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { private def apply0(plan: LogicalPlan): LogicalPlan = plan transformDown { case op @ PhysicalOperation(projects, filters, - l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _, - dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) => - val projectionRootFields = projects.flatMap(getRootFields) - val filterRootFields = filters.flatMap(getRootFields) - val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct - - // If [[requestedRootFields]] includes a nested field, continue. Otherwise, - // return [[op]] - if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) { - // 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 RootField(field, _) => StructType(Array(field)) } - .reduceLeft(_ merge _) - val dataSchemaFieldNames = dataSchema.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]] - val prunedDataSchema = - sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType] + l @ LogicalRelation(hadoopFsRelation: HadoopFsRelation, _, _, _)) + if canPruneRelation(hadoopFsRelation) => + val requestedRootFields = identifyRootFields(projects, filters) + + // If requestedRootFields includes a nested field, continue. Otherwise, + // return op + if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { + val dataSchema = hadoopFsRelation.dataSchema + val prunedDataSchema = pruneDataSchema(dataSchema, requestedRootFields) // If the data schema is different from the pruned data schema, continue. Otherwise, - // return [[op]]. We effect this comparison by counting the number of "leaf" fields in - // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields - // in [[dataSchema]]. + // return op. We effect this comparison by counting the number of "leaf" fields in + // each schemata, assuming the fields in prunedDataSchema are a subset of the fields + // in dataSchema. if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) { val prunedParquetRelation = hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession) - // We need to replace the expression ids of the pruned relation output attributes - // with the expression ids of the original relation output attributes so that - // references to the original relation's output are not broken - val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap - val prunedRelationOutput = - prunedParquetRelation - .schema - .toAttributes - .map { - case att if outputIdMap.contains(att.name) => - att.withExprId(outputIdMap(att.name)) - case att => att - } - val prunedRelation = - l.copy(relation = prunedParquetRelation, output = prunedRelationOutput) - + val prunedRelation = buildPrunedRelation(l, prunedParquetRelation) val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) - // Construct a new target for our projection by rewriting and - // including the original filters where available - val projectionChild = - if (filters.nonEmpty) { - val projectedFilters = filters.map(_.transformDown { - case projectionOverSchema(expr) => expr - }) - val newFilterCondition = projectedFilters.reduce(And) - Filter(newFilterCondition, prunedRelation) - } else { - prunedRelation - } - - // Construct the new projections of our [[Project]] by - // rewriting the original projections - val newProjects = projects.map(_.transformDown { - case projectionOverSchema(expr) => expr - }).map { case expr: NamedExpression => expr } - - logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") - logDebug(s"Pruned data schema:\n${prunedDataSchema.treeString}") - - Project(newProjects, projectionChild) + buildNewProjection(projects, filters, prunedRelation, projectionOverSchema) } else { op } @@ -122,25 +73,113 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { } } + /** + * Checks to see if the given relation is Parquet and can be pruned. + */ + private def canPruneRelation(fsRelation: HadoopFsRelation) = + fsRelation.fileFormat.isInstanceOf[ParquetFileFormat] + + /** + * Returns the set of fields from the Parquet file that the query plan needs. + */ + private def identifyRootFields(projects: Seq[NamedExpression], filters: Seq[Expression]) = { + val projectionRootFields = projects.flatMap(getRootFields) + val filterRootFields = filters.flatMap(getRootFields) + + (projectionRootFields ++ filterRootFields).distinct + } + + /** + * Builds the new output [[Project]] Spark SQL operator that has the pruned output relation. + */ + private def buildNewProjection( + projects: Seq[NamedExpression], filters: Seq[Expression], prunedRelation: LogicalRelation, + projectionOverSchema: ProjectionOverSchema) = { + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + Filter(newFilterCondition, prunedRelation) + } else { + prunedRelation + } + + // Construct the new projections of our Project by + // rewriting the original projections + val newProjects = projects.map(_.transformDown { + case projectionOverSchema(expr) => expr + }).map { case expr: NamedExpression => expr } + + if (log.isDebugEnabled) { + logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") + } + + Project(newProjects, projectionChild) + } + + /** + * Filters the schema from the given file by the requested fields. + * Schema field ordering from the file is preserved. + */ + private def pruneDataSchema( + fileDataSchema: StructType, + requestedRootFields: Seq[RootField]) = { + // 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 RootField(field, _) => StructType(Array(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] + } + + private def buildPrunedRelation( + outputRelation: LogicalRelation, + parquetRelation: HadoopFsRelation) = { + // We need to replace the expression ids of the pruned relation output attributes + // with the expression ids of the original relation output attributes so that + // references to the original relation's output are not broken + val outputIdMap = outputRelation.output.map(att => (att.name, att.exprId)).toMap + val prunedRelationOutput = + parquetRelation + .schema + .toAttributes + .map { + case att if outputIdMap.contains(att.name) => + att.withExprId(outputIdMap(att.name)) + case att => att + } + outputRelation.copy(relation = parquetRelation, output = prunedRelationOutput) + } + /** * Gets the root (aka top-level, no-parent) [[StructField]]s for the given [[Expression]]. - * When [[expr]] is an [[Attribute]], construct a field around it and indicate that that + * When expr is an [[Attribute]], construct a field around it and indicate that that * field was derived from an attribute. */ private def getRootFields(expr: Expression): Seq[RootField] = { expr match { case att: Attribute => - RootField(StructField(att.name, att.dataType, att.nullable), true) :: Nil - case SelectedField(field) => RootField(field, false) :: Nil + RootField(StructField(att.name, att.dataType, att.nullable), derivedFromAtt = true) :: Nil + case SelectedField(field) => RootField(field, derivedFromAtt = false) :: Nil case _ => expr.children.flatMap(getRootFields) } } /** - * Counts the "leaf" fields of the given [[dataType]]. Informally, this is the + * Counts the "leaf" fields of the given dataType. Informally, this is the * number of fields of non-complex data type in the tree representation of - * [[dataType]]. + * [[DataType]]. */ private def countLeaves(dataType: DataType): Int = { dataType match { @@ -153,10 +192,10 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { } /** - * 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. + * 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. */ private def sortLeftFieldsByRight(left: DataType, right: DataType): DataType = (left, right) match { @@ -179,7 +218,7 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { StructField(fieldName, sortedLeftFieldType) } StructType(sortedLeftFields) - case (left, _) => left + case _ => left } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala deleted file mode 100644 index 52911034ff069..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.execution - -import org.scalactic.Equality -import org.scalatest.Assertions - -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.types.StructType - -private[sql] trait FileSchemaPruningTest { - _: Assertions => - - private val schemaEquality = new Equality[StructType] { - override def areEqual(a: StructType, b: Any) = - b match { - case otherType: StructType => a sameType otherType - case _ => false - } - } - - protected def checkScan(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { - checkScanSchemata(df, expectedSchemaCatalogStrings: _*) - // We check here that we can execute the query without throwing an exception. The results - // themselves are irrelevant, and should be checked elsewhere as needed - df.collect() - } - - private def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { - val fileSourceScanSchemata = - df.queryExecution.executedPlan.collect { - case scan: FileSourceScanExec => scan.requiredSchema - } - assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, - s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + - s"but expected ${expectedSchemaCatalogStrings}") - fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { - case (scanSchema, expectedScanSchemaCatalogString) => - val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) - implicit val equality = schemaEquality - assert(scanSchema === expectedScanSchema) - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala similarity index 61% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala index f4ed7570ab258..05f7e3ce83880 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.planning +package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.scalatest.exceptions.TestFailedException @@ -27,36 +27,14 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ -// scalastyle:off line.size.limit class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { + private val ignoredField = StructField("col1", StringType, nullable = false) + // The test schema as a tree string, i.e. `schema.treeString` // root // |-- col1: string (nullable = false) // |-- col2: struct (nullable = true) // | |-- field1: integer (nullable = true) - // | |-- field2: array (nullable = true) - // | | |-- element: integer (containsNull = false) - // | |-- field3: array (nullable = false) - // | | |-- element: struct (containsNull = true) - // | | | |-- subfield1: integer (nullable = true) - // | | | |-- subfield2: integer (nullable = true) - // | | | |-- subfield3: array (nullable = true) - // | | | | |-- element: integer (containsNull = true) - // | |-- field4: map (nullable = true) - // | | |-- key: string - // | | |-- value: struct (valueContainsNull = false) - // | | | |-- subfield1: integer (nullable = true) - // | | | |-- subfield2: array (nullable = true) - // | | | | |-- element: integer (containsNull = false) - // | |-- field5: array (nullable = false) - // | | |-- element: struct (containsNull = true) - // | | | |-- subfield1: struct (nullable = false) - // | | | | |-- subsubfield1: integer (nullable = true) - // | | | | |-- subsubfield2: integer (nullable = true) - // | | | |-- subfield2: struct (nullable = true) - // | | | | |-- subsubfield1: struct (nullable = true) - // | | | | | |-- subsubsubfield1: string (nullable = true) - // | | | | |-- subsubfield2: integer (nullable = true) // | |-- field6: struct (nullable = true) // | | |-- subfield1: string (nullable = false) // | | |-- subfield2: string (nullable = true) @@ -64,80 +42,12 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { // | | |-- subfield1: struct (nullable = true) // | | | |-- subsubfield1: integer (nullable = true) // | | | |-- subsubfield2: integer (nullable = true) - // | |-- field8: map (nullable = true) - // | | |-- key: string - // | | |-- value: array (valueContainsNull = false) - // | | | |-- element: struct (containsNull = true) - // | | | | |-- subfield1: integer (nullable = true) - // | | | | |-- subfield2: array (nullable = true) - // | | | | | |-- element: integer (containsNull = false) // | |-- field9: map (nullable = true) // | | |-- key: string // | | |-- value: integer (valueContainsNull = false) - // |-- col3: array (nullable = false) - // | |-- element: struct (containsNull = false) - // | | |-- field1: struct (nullable = true) - // | | | |-- subfield1: integer (nullable = false) - // | | | |-- subfield2: integer (nullable = true) - // | | |-- field2: map (nullable = true) - // | | | |-- key: string - // | | | |-- value: integer (valueContainsNull = false) - // |-- col4: map (nullable = false) - // | |-- key: string - // | |-- value: struct (valueContainsNull = false) - // | | |-- field1: struct (nullable = true) - // | | | |-- subfield1: integer (nullable = false) - // | | | |-- subfield2: integer (nullable = true) - // | | |-- field2: map (nullable = true) - // | | | |-- key: string - // | | | |-- value: integer (valueContainsNull = false) - // |-- col5: array (nullable = true) - // | |-- element: map (containsNull = true) - // | | |-- key: string - // | | |-- value: struct (valueContainsNull = false) - // | | | |-- field1: struct (nullable = true) - // | | | | |-- subfield1: integer (nullable = true) - // | | | | |-- subfield2: integer (nullable = true) - // |-- col6: map (nullable = true) - // | |-- key: string - // | |-- value: array (valueContainsNull = true) - // | | |-- element: struct (containsNull = false) - // | | | |-- field1: struct (nullable = true) - // | | | | |-- subfield1: integer (nullable = true) - // | | | | |-- subfield2: integer (nullable = true) - // |-- col7: array (nullable = true) - // | |-- element: struct (containsNull = true) - // | | |-- field1: integer (nullable = false) - // | | |-- field2: struct (nullable = true) - // | | | |-- subfield1: integer (nullable = false) - // | | |-- field3: array (nullable = true) - // | | | |-- element: struct (containsNull = true) - // | | | | |-- subfield1: integer (nullable = false) - // |-- col8: array (nullable = true) - // | |-- element: struct (containsNull = true) - // | | |-- field1: array (nullable = false) - // | | | |-- element: integer (containsNull = false) - private val schema = - StructType( - StructField("col1", StringType, nullable = false) :: + private val nestedComplex = StructType(ignoredField :: StructField("col2", StructType( StructField("field1", IntegerType) :: - StructField("field2", ArrayType(IntegerType, containsNull = false)) :: - StructField("field3", ArrayType(StructType( - StructField("subfield1", IntegerType) :: - StructField("subfield2", IntegerType) :: - StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: - StructField("field4", MapType(StringType, StructType( - StructField("subfield1", IntegerType) :: - StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: - StructField("field5", ArrayType(StructType( - StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: - StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: - StructField("subfield2", StructType( - StructField("subsubfield1", StructType( - StructField("subsubsubfield1", StringType) :: Nil)) :: - StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) :: StructField("field6", StructType( StructField("subfield1", StringType, nullable = false) :: StructField("subfield2", StringType) :: Nil)) :: @@ -145,158 +55,178 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { StructField("subfield1", StructType( StructField("subsubfield1", IntegerType) :: StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: - StructField("field8", MapType(StringType, ArrayType(StructType( - StructField("subfield1", IntegerType) :: - StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) :: - StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) :: - StructField("col3", ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: - StructField("subfield2", IntegerType) :: Nil)) :: - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) :: - StructField("col4", MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: - StructField("subfield2", IntegerType) :: Nil)) :: - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) :: - StructField("col5", ArrayType(MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: - StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) :: - StructField("col6", MapType(StringType, ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: - StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) :: - StructField("col7", ArrayType(StructType( - StructField("field1", IntegerType, nullable = false) :: - StructField("field2", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: - StructField("field3", ArrayType(StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) :: - StructField("col8", ArrayType(StructType( - StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil) - - private val testRelation = LocalRelation(schema.toAttributes) + StructField("field9", + MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) :: Nil) test("SelectedField should not match an attribute reference") { - assertResult(None)(unapplySelect("col1")) - assertResult(None)(unapplySelect("col1 as foo")) - assertResult(None)(unapplySelect("col2")) + val testRelation = LocalRelation(nestedComplex.toAttributes) + assertResult(None)(unapplySelect("col1", testRelation)) + assertResult(None)(unapplySelect("col1 as foo", testRelation)) + assertResult(None)(unapplySelect("col2", testRelation)) } - info("For a relation with schema\n" + indent(schema.treeString)) + // |-- col1: string (nullable = false) + // |-- col2: struct (nullable = true) + // | |-- field2: array (nullable = true) + // | | |-- element: integer (containsNull = false) + // | |-- field3: array (nullable = false) + // | | |-- element: struct (containsNull = true) + // | | | |-- subfield1: integer (nullable = true) + // | | | |-- subfield2: integer (nullable = true) + // | | | |-- subfield3: array (nullable = true) + // | | | | |-- element: integer (containsNull = true) + private val structOfArray = StructType(ignoredField :: + StructField("col2", StructType( + StructField("field2", ArrayType(IntegerType, containsNull = false)) :: + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: + StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) + :: Nil)) + :: Nil) - testSelect("col2.field2", "col2.field2[0] as foo") { + testSelect(structOfArray, "col2.field2", "col2.field2[0] as foo") { StructField("col2", StructType( StructField("field2", ArrayType(IntegerType, containsNull = false)) :: Nil)) } - testSelect("col2.field9", "col2.field9['foo'] as foo") { + testSelect(nestedComplex, "col2.field9", "col2.field9['foo'] as foo") { StructField("col2", StructType( StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) } - testSelect("col2.field3.subfield3", "col2.field3[0].subfield3 as foo", + testSelect(structOfArray, "col2.field3.subfield3", "col2.field3[0].subfield3 as foo", "col2.field3.subfield3[0] as foo", "col2.field3[0].subfield3[0] as foo") { StructField("col2", StructType( StructField("field3", ArrayType(StructType( StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: Nil)) } - testSelect("col2.field3.subfield1") { + testSelect(structOfArray, "col2.field3.subfield1") { StructField("col2", StructType( StructField("field3", ArrayType(StructType( StructField("subfield1", IntegerType) :: Nil)), nullable = false) :: Nil)) } - testSelect("col2.field5.subfield1") { + // |-- col1: string (nullable = false) + // |-- col2: struct (nullable = true) + // | |-- field4: map (nullable = true) + // | | |-- key: string + // | | |-- value: struct (valueContainsNull = false) + // | | | |-- subfield1: integer (nullable = true) + // | | | |-- subfield2: array (nullable = true) + // | | | | |-- element: integer (containsNull = false) + // | |-- field8: map (nullable = true) + // | | |-- key: string + // | | |-- value: array (valueContainsNull = false) + // | | | |-- element: struct (containsNull = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: array (nullable = true) + // | | | | | |-- element: integer (containsNull = false) + private val structWithMap = StructType( + ignoredField :: StructField("col2", StructType( - StructField("field5", ArrayType(StructType( - StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: - StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) - } - - testSelect("col3.field1.subfield1") { - StructField("col3", ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), containsNull = false), nullable = false) - } - - testSelect("col3.field2['foo'] as foo") { - StructField("col3", ArrayType(StructType( - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) - } - - testSelect("col4['foo'].field1.subfield1 as foo") { - StructField("col4", MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), valueContainsNull = false), nullable = false) - } - - testSelect("col4['foo'].field2['bar'] as foo") { - StructField("col4", MapType(StringType, StructType( - StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) - } + StructField("field4", MapType(StringType, StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil + ), valueContainsNull = false)) :: + StructField("field8", MapType(StringType, ArrayType(StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil) + ), valueContainsNull = false)) :: Nil + )) :: Nil + ) - testSelect("col5[0]['foo'].field1.subfield1 as foo") { - StructField("col5", ArrayType(MapType(StringType, StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + testSelect(structWithMap, "col2.field4['foo'].subfield1 as foo") { + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil)) } - testSelect("col6['foo'][0].field1.subfield1 as foo") { - StructField("col6", MapType(StringType, ArrayType(StructType( - StructField("field1", StructType( - StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false))) + testSelect(structWithMap, + "col2.field4['foo'].subfield2 as foo", "col2.field4['foo'].subfield2[0] as foo") { + StructField("col2", StructType( + StructField("field4", MapType(StringType, StructType( + StructField("subfield2", ArrayType(IntegerType, containsNull = false)) + :: Nil), valueContainsNull = false)) :: Nil)) } - testSelect("col2.field5.subfield1.subsubfield1") { + // |-- col1: string (nullable = false) + // |-- col2: struct (nullable = true) + // | |-- field5: array (nullable = false) + // | | |-- element: struct (containsNull = true) + // | | | |-- subfield1: struct (nullable = false) + // | | | | |-- subsubfield1: integer (nullable = true) + // | | | | |-- subsubfield2: integer (nullable = true) + // | | | |-- subfield2: struct (nullable = true) + // | | | | |-- subsubfield1: struct (nullable = true) + // | | | | | |-- subsubsubfield1: string (nullable = true) + // | | | | |-- subsubfield2: integer (nullable = true) + private val structWithArray = StructType( + ignoredField :: StructField("col2", StructType( StructField("field5", ArrayType(StructType( StructField("subfield1", StructType( - StructField("subsubfield1", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil)) - } + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: + StructField("subfield2", StructType( + StructField("subsubfield1", StructType( + StructField("subsubsubfield1", StringType) :: Nil)) :: + StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) :: Nil) + ) :: Nil + ) - testSelect("col2.field5.subfield2.subsubfield1.subsubsubfield1") { + testSelect(structWithArray, "col2.field5.subfield1") { StructField("col2", StructType( StructField("field5", ArrayType(StructType( - StructField("subfield2", StructType( - StructField("subsubfield1", StructType( - StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) :: Nil)), nullable = false) :: Nil)) + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: + StructField("subsubfield2", IntegerType) :: Nil), nullable = false) + :: Nil)), nullable = false) :: Nil)) } - testSelect("col2.field4['foo'].subfield1 as foo") { + testSelect(structWithArray, "col2.field5.subfield1.subsubfield1") { StructField("col2", StructType( - StructField("field4", MapType(StringType, StructType( - StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil)) + StructField("field5", ArrayType(StructType( + StructField("subfield1", StructType( + StructField("subsubfield1", IntegerType) :: Nil), nullable = false) + :: Nil)), nullable = false) :: Nil)) } - testSelect("col2.field4['foo'].subfield2 as foo", "col2.field4['foo'].subfield2[0] as foo") { + testSelect(structWithArray, "col2.field5.subfield2.subsubfield1.subsubsubfield1") { StructField("col2", StructType( - StructField("field4", MapType(StringType, StructType( - StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: Nil)) + StructField("field5", ArrayType(StructType( + StructField("subfield2", StructType( + StructField("subsubfield1", StructType( + StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) + :: Nil)), nullable = false) :: Nil)) } - testSelect("col2.field8['foo'][0].subfield1 as foo") { + testSelect(structWithMap, "col2.field8['foo'][0].subfield1 as foo") { StructField("col2", StructType( StructField("field8", MapType(StringType, ArrayType(StructType( StructField("subfield1", IntegerType) :: Nil)), valueContainsNull = false)) :: Nil)) } - testSelect("col2.field1") { + testSelect(nestedComplex, "col2.field1") { StructField("col2", StructType( StructField("field1", IntegerType) :: Nil)) } - testSelect("col2.field6") { + testSelect(nestedComplex, "col2.field6") { StructField("col2", StructType( StructField("field6", StructType( StructField("subfield1", StringType, nullable = false) :: StructField("subfield2", StringType) :: Nil)) :: Nil)) } - testSelect("col2.field7.subfield1") { + testSelect(nestedComplex, "col2.field6.subfield1") { + StructField("col2", StructType( + StructField("field6", StructType( + StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil)) + } + + testSelect(nestedComplex, "col2.field7.subfield1") { StructField("col2", StructType( StructField("field7", StructType( StructField("subfield1", StructType( @@ -304,32 +234,165 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: Nil)) } - testSelect("col2.field6.subfield1") { - StructField("col2", StructType( - StructField("field6", StructType( - StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil)) + // |-- col1: string (nullable = false) + // |-- col3: array (nullable = false) + // | |-- element: struct (containsNull = false) + // | | |-- field1: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | | |-- subfield2: integer (nullable = true) + // | | |-- field2: map (nullable = true) + // | | | |-- key: string + // | | | |-- value: integer (valueContainsNull = false) + private val arrayWithStructAndMap = StructType(Array( + StructField("col3", ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: + StructField("subfield2", IntegerType) :: Nil)) :: + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) + :: Nil), containsNull = false), nullable = false) + )) + + testSelect(arrayWithStructAndMap, "col3.field1.subfield1") { + StructField("col3", ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) + :: Nil), containsNull = false), nullable = false) + } + + testSelect(arrayWithStructAndMap, "col3.field2['foo'] as foo") { + StructField("col3", ArrayType(StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) + :: Nil), containsNull = false), nullable = false) + } + + // |-- col1: string (nullable = false) + // |-- col4: map (nullable = false) + // | |-- key: string + // | |-- value: struct (valueContainsNull = false) + // | | |-- field1: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | | |-- subfield2: integer (nullable = true) + // | | |-- field2: map (nullable = true) + // | | | |-- key: string + // | | | |-- value: integer (valueContainsNull = false) + private val col4 = StructType(Array(ignoredField, + StructField("col4", MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: + StructField("subfield2", IntegerType) :: Nil)) :: + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) + :: Nil), valueContainsNull = false), nullable = false) + )) + + testSelect(col4, "col4['foo'].field1.subfield1 as foo") { + StructField("col4", MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) + :: Nil), valueContainsNull = false), nullable = false) } - testSelect("col7.field1", "col7[0].field1 as foo", "col7.field1[0] as foo") { + testSelect(col4, "col4['foo'].field2['bar'] as foo") { + StructField("col4", MapType(StringType, StructType( + StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) + :: Nil), valueContainsNull = false), nullable = false) + } + + // |-- col1: string (nullable = false) + // |-- col5: array (nullable = true) + // | |-- element: map (containsNull = true) + // | | |-- key: string + // | | |-- value: struct (valueContainsNull = false) + // | | | |-- field1: struct (nullable = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: integer (nullable = true) + private val arrayOfStruct = StructType(Array(ignoredField, + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + )) + + testSelect(arrayOfStruct, "col5[0]['foo'].field1.subfield1 as foo") { + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + } + + // |-- col1: string (nullable = false) + // |-- col6: map (nullable = true) + // | |-- key: string + // | |-- value: array (valueContainsNull = true) + // | | |-- element: struct (containsNull = false) + // | | | |-- field1: struct (nullable = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: integer (nullable = true) + private val mapOfArray = StructType(Array(ignoredField, + StructField("col6", MapType(StringType, ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))))) + + testSelect(mapOfArray, "col6['foo'][0].field1.subfield1 as foo") { + StructField("col6", MapType(StringType, ArrayType(StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false))) + } + + // An array with a struct with a different fields + // |-- col1: string (nullable = false) + // |-- col7: array (nullable = true) + // | |-- element: struct (containsNull = true) + // | | |-- field1: integer (nullable = false) + // | | |-- field2: struct (nullable = true) + // | | | |-- subfield1: integer (nullable = false) + // | | |-- field3: array (nullable = true) + // | | | |-- element: struct (containsNull = true) + // | | | | |-- subfield1: integer (nullable = false) + private val arrayWithMultipleFields = StructType(Array(ignoredField, + StructField("col7", ArrayType(StructType( + StructField("field1", IntegerType, nullable = false) :: + StructField("field2", StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: + StructField("field3", ArrayType(StructType( + StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))))) + + testSelect(arrayWithMultipleFields, + "col7.field1", "col7[0].field1 as foo", "col7.field1[0] as foo") { StructField("col7", ArrayType(StructType( StructField("field1", IntegerType, nullable = false) :: Nil))) } - testSelect("col7.field2.subfield1") { + testSelect(arrayWithMultipleFields, "col7.field2.subfield1") { StructField("col7", ArrayType(StructType( StructField("field2", StructType( StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil))) } - testSelect("col7.field3.subfield1") { + testSelect(arrayWithMultipleFields, "col7.field3.subfield1") { StructField("col7", ArrayType(StructType( StructField("field3", ArrayType(StructType( StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) } - testSelect("col8.field1", "col8[0].field1 as foo", "col8.field1[0] as foo", "col8[0].field1[0] as foo") { + // Array with a nested int array + // |-- col1: string (nullable = false) + // |-- col8: array (nullable = true) + // | |-- element: struct (containsNull = true) + // | | |-- field1: array (nullable = false) + // | | | |-- element: integer (containsNull = false) + private val arrayOfArray = StructType(Array(ignoredField, + StructField("col8", + ArrayType(StructType(Array(StructField("field1", + ArrayType(IntegerType, containsNull = false), nullable = false)))) + ))) + + testSelect(arrayOfArray, "col8.field1", + "col8[0].field1 as foo", + "col8.field1[0] as foo", + "col8[0].field1[0] as foo") { StructField("col8", ArrayType(StructType( - StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) + StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) + :: Nil))) } def assertResult(expected: StructField)(actual: StructField)(selectExpr: String): Unit = { @@ -350,18 +413,19 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { // Test that the given SELECT expressions prune the test schema to the single-column schema // defined by the given field - private def testSelect(selectExpr: String, otherSelectExprs: String*)(expected: StructField) { - val selectExprs = selectExpr +: otherSelectExprs + private def testSelect(inputSchema: StructType, selectExprs: String*) + (expected: StructField) { test(s"SELECT ${selectExprs.map(s => s""""$s"""").mkString(", ")} should select the schema\n" + - indent(StructType(expected :: Nil).treeString)) { + indent(StructType(expected :: Nil).treeString)) { for (selectExpr <- selectExprs) { - assertSelect(selectExpr, expected) + assertSelect(selectExpr, expected, inputSchema) } } } - private def assertSelect(expr: String, expected: StructField) = { - unapplySelect(expr) match { + private def assertSelect(expr: String, expected: StructField, inputSchema: StructType): Unit = { + val relation = LocalRelation(inputSchema.toAttributes) + unapplySelect(expr, relation) match { case Some(field) => assertResult(expected)(field)(expr) case None => @@ -373,16 +437,19 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { } } - private def unapplySelect(expr: String) = { - val parsedExpr = - CatalystSqlParser.parseExpression(expr) match { - case namedExpr: NamedExpression => namedExpr - } - val select = testRelation.select(parsedExpr) + private def unapplySelect(expr: String, relation: LocalRelation) = { + val parsedExpr = parseAsCatalystExpression(Seq(expr)).head + val select = relation.select(parsedExpr) val analyzed = select.analyze SelectedField.unapply(analyzed.expressions.head) } + private def parseAsCatalystExpression(exprs: Seq[String]) = { + exprs.map(CatalystSqlParser.parseExpression(_) match { + case namedExpr: NamedExpression => namedExpr + }) + } + // Indent every line in `string` by four spaces private def indent(string: String) = string.replaceAll("(?m)^", " ") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index d9e4696c3fa15..b70e19462d8b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -19,15 +19,18 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File -import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.execution.FileSchemaPruningTest +import org.scalactic.Equality + +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType class ParquetSchemaPruningSuite extends QueryTest with ParquetTest - with FileSchemaPruningTest with SharedSQLContext { case class FullName(first: String, middle: String, last: String) case class Contact( @@ -35,14 +38,14 @@ class ParquetSchemaPruningSuite name: FullName, address: String, pets: Int, - friends: Array[FullName] = Array(), - relatives: Map[String, FullName] = Map()) + friends: Array[FullName] = Array.empty, + relatives: Map[String, FullName] = Map.empty) val janeDoe = FullName("Jane", "X.", "Doe") val johnDoe = FullName("John", "Y.", "Doe") val susanSmith = FullName("Susan", "Z.", "Smith") - val contacts = + private val contacts = Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), relatives = Map("brother" -> johnDoe)) :: Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe)) :: Nil @@ -50,7 +53,7 @@ class ParquetSchemaPruningSuite case class Name(first: String, last: String) case class BriefContact(id: Int, name: Name, address: String) - val briefContacts = + private val briefContacts = BriefContact(2, Name("Janet", "Jones"), "567 Maple Drive") :: BriefContact(3, Name("Jim", "Jones"), "6242 Ash Street") :: Nil @@ -65,10 +68,10 @@ class ParquetSchemaPruningSuite case class BriefContactWithDataPartitionColumn(id: Int, name: Name, address: String, p: Int) - val contactsWithDataPartitionColumn = + private val contactsWithDataPartitionColumn = contacts.map { case Contact(id, name, address, pets, friends, relatives) => ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, 1) } - val briefContactsWithDataPartitionColumn = + private val briefContactsWithDataPartitionColumn = briefContacts.map { case BriefContact(id, name, address) => BriefContactWithDataPartitionColumn(id, name, address, 2) } @@ -161,10 +164,10 @@ class ParquetSchemaPruningSuite } withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - test(s"Parquet-mr reader - without partition data column - $testName") { + test(s"Native Parquet reader - without partition data column - $testName") { withContacts(testThunk) } - test(s"Parquet-mr reader - with partition data column - $testName") { + test(s"Native Parquet reader - with partition data column - $testName") { withContactsWithDataPartitionColumn(testThunk) } } @@ -195,4 +198,35 @@ class ParquetSchemaPruningSuite testThunk } } + + private val schemaEquality = new Equality[StructType] { + override def areEqual(a: StructType, b: Any): Boolean = + b match { + case otherType: StructType => a.sameType(otherType) + case _ => false + } + } + + protected def checkScan(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + checkScanSchemata(df, expectedSchemaCatalogStrings: _*) + // We check here that we can execute the query without throwing an exception. The results + // themselves are irrelevant, and should be checked elsewhere as needed + df.collect() + } + + private def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan.requiredSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings") + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } } From 61c7937e39fce02d9ae7385d9adaaf7bc913b7ed Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 20 Aug 2018 10:28:52 -0700 Subject: [PATCH 21/24] Update terminology for parquet-mr reader in ParquetSchemaPruningSuite.scala to "Parquet-mr" when running with Spark's vectorized reader turned off --- .../datasources/parquet/ParquetSchemaPruningSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index b70e19462d8b3..f43181c1e7505 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -164,10 +164,10 @@ class ParquetSchemaPruningSuite } withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - test(s"Native Parquet reader - without partition data column - $testName") { + test(s"Parquet-mr reader - without partition data column - $testName") { withContacts(testThunk) } - test(s"Native Parquet reader - with partition data column - $testName") { + test(s"Parquet-mr reader - with partition data column - $testName") { withContactsWithDataPartitionColumn(testThunk) } } From 97b3a51d478f19890ded73aa78d94c055a9f144c Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 20 Aug 2018 19:28:01 -0700 Subject: [PATCH 22/24] Handle differences in letter case in columns and fields between query projections and filters, and the underlying parquet file schema --- .../parquet/ParquetSchemaPruning.scala | 40 ++++++++++-- .../parquet/ParquetSchemaPruningSuite.scala | 63 +++++++++++++++++++ 2 files changed, 97 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index 15b8615ec365a..6a46b5f8edc54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, Expression, NamedExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -45,7 +45,9 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { case op @ PhysicalOperation(projects, filters, l @ LogicalRelation(hadoopFsRelation: HadoopFsRelation, _, _, _)) if canPruneRelation(hadoopFsRelation) => - val requestedRootFields = identifyRootFields(projects, filters) + val (normalizedProjects, normalizedFilters) = + normalizeAttributeRefNames(l, projects, filters) + val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) // If requestedRootFields includes a nested field, continue. Otherwise, // return op @@ -64,7 +66,8 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { val prunedRelation = buildPrunedRelation(l, prunedParquetRelation) val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) - buildNewProjection(projects, filters, prunedRelation, projectionOverSchema) + buildNewProjection(normalizedProjects, normalizedFilters, prunedRelation, + projectionOverSchema) } else { op } @@ -79,6 +82,27 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { private def canPruneRelation(fsRelation: HadoopFsRelation) = fsRelation.fileFormat.isInstanceOf[ParquetFileFormat] + /** + * Normalizes the names of the attribute references in the given projects and filters to reflect + * the names in the given logical relation. This makes it possible to compare attributes and + * fields by name. Returns a tuple with the normalized projects and filters, respectively. + */ + private def normalizeAttributeRefNames( + logicalRelation: LogicalRelation, + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { + val normalizedAttNameMap = logicalRelation.output.map(att => (att.exprId, att.name)).toMap + val normalizedProjects = projects.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }).map { case expr: NamedExpression => expr } + val normalizedFilters = filters.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + (normalizedProjects, normalizedFilters) + } + /** * Returns the set of fields from the Parquet file that the query plan needs. */ @@ -142,15 +166,19 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { sortLeftFieldsByRight(mergedDataSchema, fileDataSchema).asInstanceOf[StructType] } + /** + * Builds a pruned logical relation from the output of the output relation and the schema of the + * pruned base relation. + */ private def buildPrunedRelation( outputRelation: LogicalRelation, - parquetRelation: HadoopFsRelation) = { + prunedBaseRelation: HadoopFsRelation) = { // We need to replace the expression ids of the pruned relation output attributes // with the expression ids of the original relation output attributes so that // references to the original relation's output are not broken val outputIdMap = outputRelation.output.map(att => (att.name, att.exprId)).toMap val prunedRelationOutput = - parquetRelation + prunedBaseRelation .schema .toAttributes .map { @@ -158,7 +186,7 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { att.withExprId(outputIdMap(att.name)) case att => att } - outputRelation.copy(relation = parquetRelation, output = prunedRelationOutput) + outputRelation.copy(relation = prunedBaseRelation, output = prunedRelationOutput) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index f43181c1e7505..fd671c6dbd7f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -199,6 +199,69 @@ class ParquetSchemaPruningSuite } } + case class MixedCaseColumn(a: String, B: Int) + case class MixedCase(id: Int, CoL1: String, coL2: MixedCaseColumn) + + private val mixedCaseData = + MixedCase(0, "r0c1", MixedCaseColumn("abc", 1)) :: + MixedCase(1, "r1c1", MixedCaseColumn("123", 2)) :: + Nil + + testMixedCasePruning("select with exact column names") { + val query = sql("select CoL1, coL2.B from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCasePruning("select with lowercase column names") { + val query = sql("select col1, col2.b from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCasePruning("select with different-case column names") { + val query = sql("select cOL1, cOl2.b from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCasePruning("filter with different-case column names") { + val query = sql("select id from mixedcase where Col2.b = 2") + // Pruning with filters is currently unsupported. As-is, the file reader will read the id column + // and the entire coL2 struct. Once pruning with filters has been implemented we can uncomment + // this line + // checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row(1) :: Nil) + } + + private def testMixedCasePruning(testName: String)(testThunk: => Unit) { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + test(s"Spark vectorized reader - mixed-case schema - $testName") { + withMixedCaseData(testThunk) + } + } + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + test(s"Parquet-mr reader - mixed-case schema - $testName") { + withMixedCaseData(testThunk) + } + } + } + + private def withMixedCaseData(testThunk: => Unit) { + withParquetTable(mixedCaseData, "mixedcase") { + testThunk + } + } + private val schemaEquality = new Equality[StructType] { override def areEqual(a: StructType, b: Any): Boolean = b match { From 271174637f35aa5684ec6cc1938c4c8b210553d3 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Tue, 21 Aug 2018 10:27:48 -0700 Subject: [PATCH 23/24] Add test permutations to the "testMixedCasePruning" method to test case-sensitive and insensitive query parsing --- .../parquet/ParquetSchemaPruningSuite.scala | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index fd671c6dbd7f8..3b985e5d30105 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -244,13 +244,27 @@ class ParquetSchemaPruningSuite } private def testMixedCasePruning(testName: String)(testThunk: => Unit) { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - test(s"Spark vectorized reader - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.CASE_SENSITIVE.key -> "true") { + test(s"Spark vectorized reader - case-sensitive parser - mixed-case schema - $testName") { + withMixedCaseData(testThunk) + } + } + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.CASE_SENSITIVE.key -> "false") { + test(s"Parquet-mr reader - case-insensitive parser - mixed-case schema - $testName") { withMixedCaseData(testThunk) } } - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - test(s"Parquet-mr reader - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.CASE_SENSITIVE.key -> "false") { + test(s"Spark vectorized reader - case-insensitive parser - mixed-case schema - $testName") { + withMixedCaseData(testThunk) + } + } + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.CASE_SENSITIVE.key -> "true") { + test(s"Parquet-mr reader - case-sensitive parser - mixed-case schema - $testName") { withMixedCaseData(testThunk) } } From e6baf681e06e229d740af120491d1bf0f426af99 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Thu, 23 Aug 2018 15:24:26 -0700 Subject: [PATCH 24/24] Disable SQL schema pruning by default and revert changes to ParquetRowConverter.scala which appear to be unnecessary at this point in time --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/SchemaPruningTest.scala | 45 +++++++++++++++++++ .../parquet/ParquetRowConverter.scala | 22 ++++----- .../parquet/ParquetSchemaPruningSuite.scala | 2 + 4 files changed, 56 insertions(+), 15 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1557e8b98aecb..6a3b7df5e0be5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1392,7 +1392,7 @@ object SQLConf { "reading unnecessary nested column data. Currently Parquet is the only data source that " + "implements this optimization.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala new file mode 100644 index 0000000000000..68e76fc013c18 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SchemaPruningTest.scala @@ -0,0 +1,45 @@ +/* + * 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 + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED + +/** + * A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled. + * Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true. + */ +private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll { + private var originalConfSchemaPruningEnabled = false + + override protected def beforeAll(): Unit = { + originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true) + super.beforeAll() + } + + override protected def afterAll(): Unit = { + try { + super.afterAll() + } finally { + conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 8bf0f32fdf619..1199725941842 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -130,8 +130,8 @@ private[parquet] class ParquetRowConverter( extends ParquetGroupConverter(updater) with Logging { assert( - parquetType.getFieldCount <= catalystType.length, - s"""Field count of the Parquet schema is greater than the field count of the Catalyst schema: + parquetType.getFieldCount == catalystType.length, + s"""Field counts of the Parquet schema and the Catalyst schema don't match: | |Parquet schema: |$parquetType @@ -182,12 +182,10 @@ private[parquet] class ParquetRowConverter( // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { - parquetType.getFields.asScala.map { - case parquetField => - val fieldIndex = catalystType.fieldIndex(parquetField.getName) - val catalystField = catalystType(fieldIndex) - // Converted field value should be set to the `fieldIndex`-th cell of `currentRow` - newConverter(parquetField, catalystField.dataType, new RowUpdater(currentRow, fieldIndex)) + parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map { + case ((parquetFieldType, catalystField), ordinal) => + // Converted field value should be set to the `ordinal`-th cell of `currentRow` + newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) }.toArray } @@ -195,7 +193,7 @@ private[parquet] class ParquetRowConverter( override def end(): Unit = { var i = 0 - while (i < fieldConverters.length) { + while (i < currentRow.numFields) { fieldConverters(i).updater.end() i += 1 } @@ -204,12 +202,8 @@ private[parquet] class ParquetRowConverter( override def start(): Unit = { var i = 0 - while (i < fieldConverters.length) { - fieldConverters(i).updater.start() - currentRow.setNullAt(i) - i += 1 - } while (i < currentRow.numFields) { + fieldConverters(i).updater.start() currentRow.setNullAt(i) i += 1 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 3b985e5d30105..eb99654fa78f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.scalactic.Equality import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.internal.SQLConf @@ -31,6 +32,7 @@ import org.apache.spark.sql.types.StructType class ParquetSchemaPruningSuite extends QueryTest with ParquetTest + with SchemaPruningTest with SharedSQLContext { case class FullName(first: String, middle: String, last: String) case class Contact(