Skip to content

Commit f87ce05

Browse files
viiryarxin
authored andcommitted
[SPARK-13616][SQL] Let SQLBuilder convert logical plan without a project on top of it
JIRA: https://issues.apache.org/jira/browse/SPARK-13616 ## What changes were proposed in this pull request? It is possibly that a logical plan has been removed `Project` from the top of it. Or the plan doesn't has a top `Project` from the beginning because it is not necessary. Currently the `SQLBuilder` can't convert such plans back to SQL. This change is to add this feature. ## How was this patch tested? A test is added to `LogicalPlanToSQLSuite`. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #11466 from viirya/sqlbuilder-notopselect.
1 parent 9c274ac commit f87ce05

File tree

2 files changed

+63
-1
lines changed

2 files changed

+63
-1
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala

Lines changed: 22 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi
5353
case e => e
5454
}
5555

56-
val generatedSQL = toSQL(canonicalizedPlan)
56+
val generatedSQL = toSQL(canonicalizedPlan, true)
5757
logDebug(
5858
s"""Built SQL query string successfully from given logical plan:
5959
|
@@ -78,6 +78,27 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi
7878
}
7979
}
8080

81+
private def toSQL(node: LogicalPlan, topNode: Boolean): String = {
82+
if (topNode) {
83+
node match {
84+
case d: Distinct => toSQL(node)
85+
case p: Project => toSQL(node)
86+
case a: Aggregate => toSQL(node)
87+
case s: Sort => toSQL(node)
88+
case r: RepartitionByExpression => toSQL(node)
89+
case _ =>
90+
build(
91+
"SELECT",
92+
node.output.map(_.sql).mkString(", "),
93+
"FROM",
94+
toSQL(node)
95+
)
96+
}
97+
} else {
98+
toSQL(node)
99+
}
100+
}
101+
81102
private def toSQL(node: LogicalPlan): String = node match {
82103
case Distinct(p: Project) =>
83104
projectToSQL(p, isDistinct = true)

sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@ package org.apache.spark.sql.hive
1919

2020
import scala.util.control.NonFatal
2121

22+
import org.apache.spark.sql.{DataFrame, SQLContext}
23+
import org.apache.spark.sql.catalyst.plans.logical._
2224
import org.apache.spark.sql.functions._
2325
import org.apache.spark.sql.test.SQLTestUtils
2426

@@ -54,6 +56,33 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils {
5456
sql("DROP TABLE IF EXISTS t0")
5557
}
5658

59+
private def checkPlan(plan: LogicalPlan, sqlContext: SQLContext, expected: String): Unit = {
60+
val convertedSQL = try new SQLBuilder(plan, sqlContext).toSQL catch {
61+
case NonFatal(e) =>
62+
fail(
63+
s"""Cannot convert the following logical query plan back to SQL query string:
64+
|
65+
|# Original logical query plan:
66+
|${plan.treeString}
67+
""".stripMargin, e)
68+
}
69+
70+
try {
71+
checkAnswer(sql(convertedSQL), DataFrame(sqlContext, plan))
72+
} catch { case cause: Throwable =>
73+
fail(
74+
s"""Failed to execute converted SQL string or got wrong answer:
75+
|
76+
|# Converted SQL query string:
77+
|$convertedSQL
78+
|
79+
|# Original logical query plan:
80+
|${plan.treeString}
81+
""".stripMargin,
82+
cause)
83+
}
84+
}
85+
5786
private def checkHiveQl(hiveQl: String): Unit = {
5887
val df = sql(hiveQl)
5988

@@ -157,6 +186,18 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils {
157186
"SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key")
158187
}
159188

189+
test("join plan") {
190+
val expectedSql = "SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key"
191+
192+
val df1 = sqlContext.table("parquet_t1").as("x")
193+
val df2 = sqlContext.table("parquet_t1").as("y")
194+
val joinPlan = df1.join(df2).queryExecution.analyzed
195+
196+
// Make sure we have a plain Join operator without Project on top of it.
197+
assert(joinPlan.isInstanceOf[Join])
198+
checkPlan(joinPlan, sqlContext, expectedSql)
199+
}
200+
160201
test("case") {
161202
checkHiveQl("SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0")
162203
}

0 commit comments

Comments
 (0)