Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils

trait DataSourceScanExec extends LeafExecNode with CodegenSupport {
trait DataSourceScanExec extends LeafExecNode with CodegenSupport with PredicateHelper {
val relation: BaseRelation
val metastoreTableIdentifier: Option[TableIdentifier]

Expand Down Expand Up @@ -519,8 +519,18 @@ case class FileSourceScanExec(
relation,
output.map(QueryPlan.normalizeExprId(_, output)),
requiredSchema,
partitionFilters.map(QueryPlan.normalizeExprId(_, output)),
dataFilters.map(QueryPlan.normalizeExprId(_, output)),
canonicalizeFilters(partitionFilters, output),
canonicalizeFilters(dataFilters, output),
None)
}

private def canonicalizeFilters(filters: Seq[Expression], output: Seq[Attribute])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a function description?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

: Seq[Expression] = {
if (filters.nonEmpty) {
val normalizedFilters = QueryPlan.normalizeExprId(filters.reduce(And), output)
splitConjunctivePredicates(normalizedFilters)
} else {
Nil
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.apache.spark.sql.{DataFrame, QueryTest}
import org.apache.spark.sql.test.SharedSQLContext

/**
* Tests for the sameResult function for [[SparkPlan]]s.
*/
class SameResultSuite extends QueryTest with SharedSQLContext {

test("FileSourceScanExec: different orders of data filters and partition filters") {
withTempPath { path =>
val tmpDir = path.getCanonicalPath
spark.range(10)
.selectExpr("id as a", "id + 1 as b", "id + 2 as c", "id + 3 as d")
.write
.partitionBy("a", "b")
.parquet(tmpDir)
val df = spark.read.parquet(tmpDir)
// partition filters: a > 1 AND b < 9
// data filters: c > 1 AND d < 9
val plan1 = getFileSourceScanExec(df.where("a > 1 AND b < 9 AND c > 1 AND d < 9"))
val plan2 = getFileSourceScanExec(df.where("b < 9 AND a > 1 AND d < 9 AND c > 1"))
assert(plan1.sameResult(plan2))
}
}

private def getFileSourceScanExec(df: DataFrame): FileSourceScanExec = {
df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get
.asInstanceOf[FileSourceScanExec]
}
}