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 @@ -473,11 +473,11 @@ identifierComment
;

relationPrimary
: tableIdentifier sample? tableAlias #tableName
| '(' queryNoWith ')' sample? tableAlias #aliasedQuery
| '(' relation ')' sample? (AS? strictIdentifier)? #aliasedRelation
| inlineTable #inlineTableDefault2
| functionTable #tableValuedFunction
: tableIdentifier sample? tableAlias #tableName
| '(' queryNoWith ')' sample? tableAlias #aliasedQuery
| '(' relation ')' sample? tableAlias #aliasedRelation
| inlineTable #inlineTableDefault2
| functionTable #tableValuedFunction
;

inlineTable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -739,12 +739,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
/**
* Create an alias (SubqueryAlias) for a join relation. This is practically the same as
* visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
* hooks.
* hooks. We could add alias names for output columns, for example:
* {{{
* SELECT a, b, c, d FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d)
* }}}
*/
override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
plan(ctx.relation)
.optionalMap(ctx.sample)(withSample)
.optionalMap(ctx.strictIdentifier)(aliasPlan)
val relation = plan(ctx.relation).optionalMap(ctx.sample)(withSample)
mayApplyAliasPlan(ctx.tableAlias, relation)
}

/**
Expand All @@ -756,31 +758,43 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* }}}
*/
override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
val alias = if (ctx.tableAlias.strictIdentifier == null) {
val relation = plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample)
if (ctx.tableAlias.strictIdentifier == null) {
// For un-aliased subqueries, use a default alias name that is not likely to conflict with
// normal subquery names, so that parent operators can only access the columns in subquery by
// unqualified names. Users can still use this special qualifier to access columns if they
// know it, but that's not recommended.
"__auto_generated_subquery_name"
SubqueryAlias("__auto_generated_subquery_name", relation)
} else {
ctx.tableAlias.strictIdentifier.getText
}
val subquery = SubqueryAlias(alias, plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample))
if (ctx.tableAlias.identifierList != null) {
val columnAliases = visitIdentifierList(ctx.tableAlias.identifierList)
UnresolvedSubqueryColumnAliases(columnAliases, subquery)
} else {
subquery
mayApplyAliasPlan(ctx.tableAlias, relation)
}
}

/**
* Create an alias (SubqueryAlias) for a LogicalPlan.
* Create an alias ([[SubqueryAlias]]) for a [[LogicalPlan]].
*/
private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = {
SubqueryAlias(alias.getText, plan)
}

/**
* If aliases specified in a FROM clause, create a subquery alias ([[SubqueryAlias]]) and
* column aliases for a [[LogicalPlan]].
*/
private def mayApplyAliasPlan(tableAlias: TableAliasContext, plan: LogicalPlan): LogicalPlan = {
if (tableAlias.strictIdentifier != null) {
val subquery = SubqueryAlias(tableAlias.strictIdentifier.getText, plan)
if (tableAlias.identifierList != null) {
val columnNames = visitIdentifierList(tableAlias.identifierList)
UnresolvedSubqueryColumnAliases(columnNames, subquery)
} else {
subquery
}
} else {
plan
}
}

/**
* Create a Sequence of Strings for a parenthesis enclosed alias list.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
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.Cross
import org.apache.spark.sql.catalyst.plans.{Cross, Inner}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.types._

Expand Down Expand Up @@ -490,4 +490,26 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers {
Seq("Number of column aliases does not match number of columns. " +
"Number of column aliases: 5; number of columns: 4."))
}

test("SPARK-20963 Support aliases for join relations in FROM clause") {
def joinRelationWithAliases(outputNames: Seq[String]): LogicalPlan = {
val src1 = LocalRelation('id.int, 'v1.string).as("s1")
val src2 = LocalRelation('id.int, 'v2.string).as("s2")
UnresolvedSubqueryColumnAliases(
outputNames,
SubqueryAlias(
"dst",
src1.join(src2, Inner, Option(Symbol("s1.id") === Symbol("s2.id"))))
).select(star())
}
assertAnalysisSuccess(joinRelationWithAliases("col1" :: "col2" :: "col3" :: "col4" :: Nil))
assertAnalysisError(
joinRelationWithAliases("col1" :: Nil),
Seq("Number of column aliases does not match number of columns. " +
"Number of column aliases: 1; number of columns: 4."))
assertAnalysisError(
joinRelationWithAliases("col1" :: "col2" :: "col3" :: "col4" :: "col5" :: Nil),
Seq("Number of column aliases does not match number of columns. " +
"Number of column aliases: 5; number of columns: 4."))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -506,6 +506,19 @@ class PlanParserSuite extends AnalysisTest {
).select(star()))
}

test("SPARK-20963 Support aliases for join relations in FROM clause") {
val src1 = UnresolvedRelation(TableIdentifier("src1")).as("s1")
val src2 = UnresolvedRelation(TableIdentifier("src2")).as("s2")
assertEqual(
"SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d)",
UnresolvedSubqueryColumnAliases(
Seq("a", "b", "c", "d"),
SubqueryAlias(
"dst",
src1.join(src2, Inner, Option(Symbol("s1.id") === Symbol("s2.id"))))
).select(star()))
}

test("inline table") {
assertEqual("values 1, 2, 3, 4",
UnresolvedInlineTable(Seq("col1"), Seq(1, 2, 3, 4).map(x => Seq(Literal(x)))))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,3 +18,10 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d);

-- Subquery aliases in FROM clause
SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2);

-- Aliases for join relations in FROM clause
CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1);

CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2);

SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d);
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 8
-- Number of queries: 11


-- !query 0
Expand Down Expand Up @@ -69,3 +69,29 @@ SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2)
struct<col1:int,col2:int>
-- !query 7 output
1 1


-- !query 8
CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1)
-- !query 8 schema
struct<>
-- !query 8 output



-- !query 9
CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2)
-- !query 9 schema
struct<>
-- !query 9 output



-- !query 10
SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d)
-- !query 10 schema
struct<a:int,b:string,c:int,d:decimal(2,1)>
-- !query 10 output
1 a 1 8.5
2 b 2 1
3 c 3 3.2