Github user cloud-fan commented on a diff in the pull request:
https://github.com/apache/spark/pull/21320#discussion_r199538123
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala
---
@@ -0,0 +1,153 @@
+/*
+ * 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(_, _,
+ 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 }) {
+ val prunedSchema = requestedRootFields
+ .map { case RootField(field, _) => StructType(Array(field)) }
+ .reduceLeft(_ merge _)
+ val dataSchemaFieldNames = dataSchema.fieldNames.toSet
--- End diff --
dataSchema may also contains partition columns(see the doc of
`HadoopFsRelation`), is this rule prepared for this case?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]