Skip to content

[SPARK-8023] [SQL] Add a deterministic method to Expression. #6570

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 3 commits into from
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 @@ -37,6 +37,13 @@ abstract class Expression extends TreeNode[Expression] {
* - A [[Cast]] or [[UnaryMinus]] is foldable if its child is foldable
*/
def foldable: Boolean = false

/**
* Returns true when the current expression always return the same result for a specific set of
* input values.
*/
// TODO: Need to well define what are explicit input values and implicit input values.
def deterministic: Boolean = true
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouldn't this only be true if all the children are deterministic (if there are any).

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it depends on how you interpret this. The definition here is whether this expression itself is deterministic or not (i.e. assuming fixed input, is the output deterministic?)

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh I understand the TODO now... It seems better be holistic about expressing an expression's determinism. Otherwise, you leave the burden of tree traversal to the user, which could be more error prone. But I agree that either could work given proper documentation.

def nullable: Boolean
def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ abstract class RDG(seed: Long) extends LeafExpression with Serializable {
*/
@transient protected lazy val rng = new XORShiftRandom(seed + TaskContext.get().partitionId())

override def deterministic: Boolean = false

override def nullable: Boolean = false

override def dataType: DataType = DoubleType
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,8 +179,17 @@ object ColumnPruning extends Rule[LogicalPlan] {
* expressions into one single expression.
*/
object ProjectCollapsing extends Rule[LogicalPlan] {
Copy link
Contributor

Choose a reason for hiding this comment

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

ideally we should have a unit test on the optimizer rule itself also


/** Returns true if any expression in projectList is non-deterministic. */
private def hasNondeterministic(projectList: Seq[NamedExpression]): Boolean = {
projectList.exists(expr => expr.find(!_.deterministic).isDefined)
Copy link
Contributor

Choose a reason for hiding this comment

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

this won't match on rand() + 1 will this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This will match rand() + 1. find will traverse the expression tree in pre-order.

}

def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case Project(projectList1, Project(projectList2, child)) =>
// We only collapse these two Projects if the child Project's expressions are all
// deterministic.
case Project(projectList1, Project(projectList2, child))
if !hasNondeterministic(projectList2) =>
// Create a map of Aliases to their values from the child projection.
// e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)).
val aliasMap = AttributeMap(projectList2.collect {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql

import org.scalatest.Matchers._

import org.apache.spark.sql.execution.Project
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.TestSQLContext
import org.apache.spark.sql.test.TestSQLContext.implicits._
Expand Down Expand Up @@ -446,13 +447,51 @@ class ColumnExpressionSuite extends QueryTest {
}

test("rand") {
val randCol = testData.select('key, rand(5L).as("rand"))
val randCol = testData.select($"key", rand(5L).as("rand"))
randCol.columns.length should be (2)
val rows = randCol.collect()
rows.foreach { row =>
assert(row.getDouble(1) <= 1.0)
assert(row.getDouble(1) >= 0.0)
}

def checkNumProjects(df: DataFrame, expectedNumProjects: Int): Unit = {
val projects = df.queryExecution.executedPlan.collect {
case project: Project => project
}
assert(projects.size === expectedNumProjects)
}

// We first create a plan with two Projects.
// Project [rand + 1 AS rand1, rand - 1 AS rand2]
// Project [key, (Rand 5 + 1) AS rand]
// LogicalRDD [key, value]
// Because Rand function is not deterministic, the column rand is not deterministic.
// So, in the optimizer, we will not collapse Project [rand + 1 AS rand1, rand - 1 AS rand2]
// and Project [key, Rand 5 AS rand]. The final plan still has two Projects.
val dfWithTwoProjects =
testData
.select($"key", (rand(5L) + 1).as("rand"))
.select(($"rand" + 1).as("rand1"), ($"rand" - 1).as("rand2"))
checkNumProjects(dfWithTwoProjects, 2)

// Now, we add one more project rand1 - rand2 on top of the query plan.
// Since rand1 and rand2 are deterministic (they basically apply +/- to the generated
// rand value), we can collapse rand1 - rand2 to the Project generating rand1 and rand2.
// So, the plan will be optimized from ...
// Project [(rand1 - rand2) AS (rand1 - rand2)]
// Project [rand + 1 AS rand1, rand - 1 AS rand2]
// Project [key, (Rand 5 + 1) AS rand]
// LogicalRDD [key, value]
// to ...
// Project [((rand + 1 AS rand1) - (rand - 1 AS rand2)) AS (rand1 - rand2)]
// Project [key, Rand 5 AS rand]
// LogicalRDD [key, value]
val dfWithThreeProjects = dfWithTwoProjects.select($"rand1" - $"rand2")
checkNumProjects(dfWithThreeProjects, 2)
dfWithThreeProjects.collect().foreach { row =>
assert(row.getDouble(0) === 2.0 +- 0.0001)
}
}

test("randn") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,8 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre

type UDFType = UDF

override def deterministic: Boolean = isUDFDeterministic

override def nullable: Boolean = true

@transient
Expand Down Expand Up @@ -140,6 +142,8 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr
extends Expression with HiveInspectors with Logging {
type UDFType = GenericUDF

override def deterministic: Boolean = isUDFDeterministic

override def nullable: Boolean = true

@transient
Expand Down