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 @@ -115,12 +115,10 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un

override def toString: String = s"'$name"

override def sql: String = name match {
case ParserUtils.escapedIdentifier(_) | ParserUtils.qualifiedEscapedIdentifier(_, _) => name
case _ => quoteIdentifier(name)
}
override def sql: String = name
}


object UnresolvedAttribute {
/**
* Creates an [[UnresolvedAttribute]], parsing segments separated by dots ('.').
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1
struct<>
-- !query 18 output
org.apache.spark.sql.AnalysisException
cannot resolve '`db1.t1.i1`' given input columns: [mydb2.t1.i1, mydb2.t1.i1]; line 1 pos 7
cannot resolve 'db1.t1.i1' given input columns: [mydb2.t1.i1, mydb2.t1.i1]; line 1 pos 7


-- !query 19
Expand All @@ -186,7 +186,7 @@ SELECT mydb1.t1 FROM t1
struct<>
-- !query 21 output
org.apache.spark.sql.AnalysisException
cannot resolve '`mydb1.t1`' given input columns: [mydb1.t1.i1]; line 1 pos 7
cannot resolve 'mydb1.t1' given input columns: [mydb1.t1.i1]; line 1 pos 7


-- !query 22
Expand All @@ -204,7 +204,7 @@ SELECT t1 FROM mydb1.t1
struct<>
-- !query 23 output
org.apache.spark.sql.AnalysisException
cannot resolve '`t1`' given input columns: [mydb1.t1.i1]; line 1 pos 7
cannot resolve 't1' given input columns: [mydb1.t1.i1]; line 1 pos 7


-- !query 24
Expand All @@ -221,7 +221,7 @@ SELECT mydb1.t1.i1 FROM t1
struct<>
-- !query 25 output
org.apache.spark.sql.AnalysisException
cannot resolve '`mydb1.t1.i1`' given input columns: [mydb2.t1.i1]; line 1 pos 7
cannot resolve 'mydb1.t1.i1' given input columns: [mydb2.t1.i1]; line 1 pos 7


-- !query 26
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k
struct<>
-- !query 21 output
org.apache.spark.sql.AnalysisException
cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47
cannot resolve 'k' given input columns: [testdata.a, testdata.b]; line 1 pos 47


-- !query 22
Expand Down
4 changes: 2 additions & 2 deletions sql/core/src/test/resources/sql-tests/results/pivot.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ PIVOT (
struct<>
-- !query 15 output
org.apache.spark.sql.AnalysisException
cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0
cannot resolve 'year' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0


-- !query 16
Expand Down Expand Up @@ -326,7 +326,7 @@ PIVOT (
struct<>
-- !query 21 output
org.apache.spark.sql.AnalysisException
cannot resolve '`s`' given input columns: [coursesales.course, coursesales.year, coursesales.earnings]; line 4 pos 15
cannot resolve 's' given input columns: [coursesales.course, coursesales.year, coursesales.earnings]; line 4 pos 15


-- !query 22
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2
struct<>
-- !query 5 output
org.apache.spark.sql.AnalysisException
cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7
cannot resolve '(a|b)' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7


-- !query 6
Expand All @@ -81,7 +81,7 @@ SELECT SUM(`(a)`) FROM testData2
struct<>
-- !query 8 output
org.apache.spark.sql.AnalysisException
cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11
cannot resolve '(a)' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11


-- !query 9
Expand Down Expand Up @@ -301,7 +301,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)`
struct<>
-- !query 32 output
org.apache.spark.sql.AnalysisException
cannot resolve '`(a)`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38
cannot resolve '(a)' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38


-- !query 33
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d)
struct<>
-- !query 6 output
org.apache.spark.sql.AnalysisException
cannot resolve '`a`' given input columns: [c, d]; line 1 pos 7
cannot resolve 'a' given input columns: [c, d]; line 1 pos 7


-- !query 7
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ FROM various_maps
struct<>
-- !query 5 output
org.apache.spark.sql.AnalysisException
cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7
cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7


-- !query 6
Expand Down Expand Up @@ -113,7 +113,7 @@ FROM various_maps
struct<>
-- !query 8 output
org.apache.spark.sql.AnalysisException
cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7
cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7


-- !query 9
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import org.apache.hadoop.io.{LongWritable, Text}
import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat}
import org.scalatest.Matchers._

import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.NamedExpression
import org.apache.spark.sql.execution.ProjectExec
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -114,6 +115,16 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
Row("a") :: Nil)
}

test("SPARK-25769 make UnresolvedAttribute.sql escape nested columns correctly") {
Seq(
($"a.b", "a.b"),
($"`a.b`", "`a.b`"),
($"`a`.b", "a.b"),
($"`a.b`.c", "`a.b`.c")).foreach { case (columnName, sqlString) =>
assert(columnName.expr.sql === sqlString)
}
}

test("alias and name") {
val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList")
assert(df.select(df("a").as("b")).columns.head === "b")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2067,7 +2067,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex3 = intercept[AnalysisException] {
df.selectExpr("transform(a, x -> x)")
}
assert(ex3.getMessage.contains("cannot resolve '`a`'"))
assert(ex3.getMessage.contains("cannot resolve 'a'"))
}

test("map_filter") {
Expand Down Expand Up @@ -2118,7 +2118,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex4 = intercept[AnalysisException] {
df.selectExpr("map_filter(a, (k, v) -> k > v)")
}
assert(ex4.getMessage.contains("cannot resolve '`a`'"))
assert(ex4.getMessage.contains("cannot resolve 'a'"))
}

test("filter function - array for primitive type not containing null") {
Expand Down Expand Up @@ -2219,7 +2219,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex4 = intercept[AnalysisException] {
df.selectExpr("filter(a, x -> x)")
}
assert(ex4.getMessage.contains("cannot resolve '`a`'"))
assert(ex4.getMessage.contains("cannot resolve 'a'"))
}

test("exists function - array for primitive type not containing null") {
Expand Down Expand Up @@ -2320,7 +2320,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex4 = intercept[AnalysisException] {
df.selectExpr("exists(a, x -> x)")
}
assert(ex4.getMessage.contains("cannot resolve '`a`'"))
assert(ex4.getMessage.contains("cannot resolve 'a'"))
}

test("aggregate function - array for primitive type not containing null") {
Expand Down Expand Up @@ -2446,7 +2446,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex5 = intercept[AnalysisException] {
df.selectExpr("aggregate(a, 0, (acc, x) -> x)")
}
assert(ex5.getMessage.contains("cannot resolve '`a`'"))
assert(ex5.getMessage.contains("cannot resolve 'a'"))
}

test("map_zip_with function - map of primitive types") {
Expand Down Expand Up @@ -2831,7 +2831,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
val ex4 = intercept[AnalysisException] {
df.selectExpr("zip_with(a1, a, (acc, x) -> x)")
}
assert(ex4.getMessage.contains("cannot resolve '`a`'"))
assert(ex4.getMessage.contains("cannot resolve 'a'"))
}

private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -294,7 +294,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext {
val df = Seq((1, "1")).toDF("key", "value")
val e = intercept[AnalysisException](
df.select($"key", count("invalid").over()))
assert(e.message.contains("cannot resolve '`invalid`' given input columns: [key, value]"))
assert(e.message.contains("cannot resolve 'invalid' given input columns: [key, value]"))
}

test("numerical aggregate functions on string column") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -268,17 +268,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
e = intercept[AnalysisException] {
ds.select(expr("`(_1|_2)`").as[Int])
}.getMessage
assert(e.contains("cannot resolve '`(_1|_2)`'"))
assert(e.contains("cannot resolve '(_1|_2)'"))

e = intercept[AnalysisException] {
ds.select(ds("`(_1)?+.+`"))
}.getMessage
assert(e.contains("Cannot resolve column name \"`(_1)?+.+`\""))

e = intercept[AnalysisException] {
ds.select(ds("`(_1|_2)`"))
ds.select(ds("(_1|_2)"))
}.getMessage
assert(e.contains("Cannot resolve column name \"`(_1|_2)`\""))
assert(e.contains("Cannot resolve column name \"(_1|_2)\""))
}

withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") {
Expand Down Expand Up @@ -709,7 +709,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
val e = intercept[AnalysisException] {
ds.as[ClassData2]
}
assert(e.getMessage.contains("cannot resolve '`c`' given input columns: [a, b]"), e.getMessage)
assert(e.getMessage.contains("cannot resolve 'c' given input columns: [a, b]"), e.getMessage)
}

test("runtime nullability check") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1143,7 +1143,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
|order by struct.a, struct.b
|""".stripMargin)
}
assert(error.message contains "cannot resolve '`struct.a`' given input columns: [a, b]")
assert(error.message contains "cannot resolve 'struct.a' given input columns: [a, b]")

}

Expand Down Expand Up @@ -2702,7 +2702,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {

val e = intercept[AnalysisException](sql("SELECT v.i from (SELECT i FROM v)"))
assert(e.message ==
"cannot resolve '`v.i`' given input columns: [__auto_generated_subquery_name.i]")
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 the problem here is the out-most backticks, do you know where we add it?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The out-most backticks is added in
case _, in case class UnresolvedAttribute(nameParts: Seq[String])

  override def sql: String = name match {
    case ParserUtils.escapedIdentifier(_) | ParserUtils.qualifiedEscapedIdentifier(_, _) => name
    case _ => quoteIdentifier(name)
  }

the nameParts is a Seq of "v" "i", name is v.i

"cannot resolve 'v.i' given input columns: [__auto_generated_subquery_name.i]")

checkAnswer(sql("SELECT __auto_generated_subquery_name.i from (SELECT i FROM v)"), Row(1))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -876,7 +876,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
withTempView("t") {
Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t")
val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)"))
assert(e.message.contains("cannot resolve '`a`' given input columns: [t.i, t.j]"))
assert(e.message.contains("cannot resolve 'a' given input columns: [t.i, t.j]"))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,12 +141,12 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton {

checkSQL(
TimeAdd('a, interval),
"`a` + interval 1 days"
"a + interval 1 days"
)

checkSQL(
TimeSub('a, interval),
"`a` - interval 1 days"
"a - interval 1 days"
)
}
}