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
2 changes: 2 additions & 0 deletions docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ license: |

- In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`.

- In Spark 3.2, the auto-generated `Cast` (such as those added by type coercion rules) will be stripped when generating column alias names. E.g., `sql("SELECT floor(1)").columns` will be `FLOOR(1)` instead of `FLOOR(CAST(1 AS DOUBLE))`.

## Upgrading from Spark SQL 3.0 to 3.1

- In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`.
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/sql/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ class SQLContext(object):
>>> df.createOrReplaceTempView("allTypes")
>>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a '
... 'from allTypes where b and i > 0').collect()
[Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \
[Row((i + 1)=2, (d + 1)=2.0, (NOT b)=False, list[1]=2, \
dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)]
>>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect()
[(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/sql/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ class SparkSession(SparkConversionMixin):
>>> df.createOrReplaceTempView("allTypes")
>>> spark.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a '
... 'from allTypes where b and i > 0').collect()
[Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \
[Row((i + 1)=2, (d + 1)=2.0, (NOT b)=False, list[1]=2, \
dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)]
>>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect()
[(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,9 @@ package object dsl {
if (expr.resolved && expr.dataType.sameType(to)) {
expr
} else {
Cast(expr, to)
val cast = Cast(expr, to)
cast.setTagValue(Cast.USER_SPECIFIED_CAST, true)
cast
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion}
import org.apache.spark.sql.catalyst.expressions.Cast.{forceNullable, resolvableNullability}
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
Expand All @@ -38,6 +39,11 @@ import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper}

object Cast {

/**
* A tag to decide if a CAST is specified by user.
*/
val USER_SPECIFIED_CAST = new TreeNodeTag[Boolean]("user_specified_cast")

/**
* Returns true iff we can cast `from` type to `to` type.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1595,7 +1595,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) {
val rawDataType = typedVisit[DataType](ctx.dataType())
val dataType = CharVarcharUtils.replaceCharVarcharWithStringForCast(rawDataType)
Cast(expression(ctx.expression), dataType)
val cast = Cast(expression(ctx.expression), dataType)
cast.setTagValue(Cast.USER_SPECIFIED_CAST, true)
cast
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,8 @@ package object util extends Logging {
PrettyAttribute(usePrettyExpression(e.child) + "." + e.field.name, e.dataType)
case r: RuntimeReplaceable =>
PrettyAttribute(r.mkString(r.exprsReplaced.map(toPrettySQL)), r.dataType)
case c: CastBase if !c.getTagValue(Cast.USER_SPECIFIED_CAST).getOrElse(false) =>
PrettyAttribute(usePrettyExpression(c.child).sql, c.dataType)
Copy link
Member

Choose a reason for hiding this comment

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

So the purpose is to strip Cast only from toPrettySQL? Only for changing column name but the actual Cast expression still works?

Copy link
Member

Choose a reason for hiding this comment

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

Could you elaborate it clearly in the PR 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.

Yes, it's the only place which we changed. Update the description.

}

def quoteIdentifier(name: String): String = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.analysis

import scala.collection.mutable.ArrayBuffer

import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project}
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType}

class ResolveAliasesSuite extends AnalysisTest {

private lazy val t1 = LocalRelation("a".attr.int)
private lazy val t2 = LocalRelation("b".attr.long)

private def checkAliasName(plan: LogicalPlan, expected: String): Unit = {
val analyzed = getAnalyzer.execute(plan)
val actual = analyzed.find(_.isInstanceOf[Project]).get.asInstanceOf[Project]
.projectList.head.asInstanceOf[Alias].name
assert(actual == expected)
}

private def checkSubqueryAliasName(plan: LogicalPlan, expected: String): Unit = {
val analyzed = getAnalyzer.execute(plan)
val subqueryExpression = new ArrayBuffer[SubqueryExpression]()
analyzed.transformExpressions {
case e: SubqueryExpression =>
subqueryExpression.append(e)
e
}
assert(subqueryExpression.length == 1)
val actual = subqueryExpression.head.plan.find(_.isInstanceOf[Project]).get
.asInstanceOf[Project].projectList.head.asInstanceOf[Alias].name
assert(actual == expected)
}

test("SPARK-33989: test unary expression") {
checkAliasName(t1.select(Floor(Literal(null))), "FLOOR(NULL)")
checkAliasName(t1.select(Floor("a".attr)), "FLOOR(a)")
checkAliasName(t1.select(Floor("a".attr.cast(DoubleType))), "FLOOR(CAST(a AS DOUBLE))")
}

test("SPARK-33989: test binary expression") {
checkAliasName(t1.select(EqualTo("a".attr, Literal(null))), "(a = NULL)")
checkAliasName(t1.select(EqualTo("a".attr.cast(LongType), Literal(1))),
"(CAST(a AS BIGINT) = 1)")
checkAliasName(t1.select(EqualTo("a".attr.cast(LongType), Literal("2").cast(LongType))),
"(CAST(a AS BIGINT) = CAST(2 AS BIGINT))")
}

test("SPARK-33989: test nested expression") {
checkAliasName(t1.select(StringSplit("a".attr + 1, ",", Literal(-1))),
"split((a + 1), ,, -1)")
checkAliasName(t1.select(StringSplit(("a".attr + 1).cast(StringType), ",", Literal(-1))),
"split(CAST((a + 1) AS STRING), ,, -1)")
}

test("SPARK-33989: test subquery expression") {
checkSubqueryAliasName(
t1.select(ScalarSubquery(t2.select(EqualTo("b".attr, Literal(null))))),
"(b = NULL)")
checkSubqueryAliasName(
t1.select(ScalarSubquery(t2.select(EqualTo("b".attr.cast(IntegerType), Literal(1))))),
"(CAST(b AS INT) = 1)")
}
}
4 changes: 3 additions & 1 deletion sql/core/src/main/scala/org/apache/spark/sql/Column.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1186,7 +1186,9 @@ class Column(val expr: Expression) extends Logging {
* @since 1.3.0
*/
def cast(to: DataType): Column = withExpr {
Cast(expr, CharVarcharUtils.replaceCharVarcharWithStringForCast(to))
val cast = Cast(expr, CharVarcharUtils.replaceCharVarcharWithStringForCast(to))
cast.setTagValue(Cast.USER_SPECIFIED_CAST, true)
cast
}

/**
Expand Down
Loading