spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From wenc...@apache.org
Subject spark git commit: [SPARK-13974][SQL] sub-query names do not need to be globally unique while generate SQL
Date Fri, 18 Mar 2016 01:30:44 GMT
Repository: spark
Updated Branches:
  refs/heads/master 1614485fd -> 453455c47


[SPARK-13974][SQL] sub-query names do not need to be globally unique while generate SQL

## What changes were proposed in this pull request?

We only need to make sub-query names unique every time we generate a SQL string, but not all
the time. This PR moves the `newSubqueryName` method to `class SQLBuilder` and remove `object
SQLBuilder`.

also addressed 2 minor comments in https://github.com/apache/spark/pull/11696

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11783 from cloud-fan/tmp.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/453455c4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/453455c4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/453455c4

Branch: refs/heads/master
Commit: 453455c479308badeeef2ad000577800982b1d4f
Parents: 1614485
Author: Wenchen Fan <wenchen@databricks.com>
Authored: Fri Mar 18 09:30:36 2016 +0800
Committer: Wenchen Fan <wenchen@databricks.com>
Committed: Fri Mar 18 09:30:36 2016 +0800

----------------------------------------------------------------------
 .../org/apache/spark/sql/hive/SQLBuilder.scala  | 21 +++++++++-----------
 1 file changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/453455c4/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
index 05dfad2..9fa8474 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
@@ -54,6 +54,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
 
   def this(df: DataFrame) = this(df.queryExecution.analyzed, df.sqlContext)
 
+  private val nextSubqueryId = new AtomicLong(0)
+  private def newSubqueryName(): String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}"
+
   def toSQL: String = {
     val canonicalizedPlan = Canonicalizer.execute(logicalPlan)
     val outputNames = logicalPlan.output.map(_.name)
@@ -64,7 +67,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
     val finalName = if (qualifiers.length == 1) {
       qualifiers.head
     } else {
-      SQLBuilder.newSubqueryName
+      newSubqueryName()
     }
 
     // Canonicalizer will remove all naming information, we should add it back by adding
an extra
@@ -254,7 +257,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
   }
 
   private def generateToSQL(g: Generate): String = {
-    val columnAliases = g.generatorOutput.map(_.sql).mkString(",")
+    val columnAliases = g.generatorOutput.map(_.sql).mkString(", ")
 
     val childSQL = if (g.child == OneRowRelation) {
       // This only happens when we put UDTF in project list and there is no FROM clause.
Because we
@@ -262,8 +265,8 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
       // after FROM clause, so that we can generate a valid LATERAL VIEW SQL string.
       // For example, if the original SQL is: "SELECT EXPLODE(ARRAY(1, 2))", we will convert
in to
       // LATERAL VIEW format, and generate:
-      // SELECT col FROM (SELECT 1) sub-q0 LATERAL VIEW EXPLODE(ARRAY(1, 2)) sub_q1 AS col
-      s"(SELECT 1) ${SQLBuilder.newSubqueryName}"
+      // SELECT col FROM (SELECT 1) sub_q0 LATERAL VIEW EXPLODE(ARRAY(1, 2)) sub_q1 AS col
+      s"(SELECT 1) ${newSubqueryName()}"
     } else {
       toSQL(g.child)
     }
@@ -283,7 +286,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
       "LATERAL VIEW",
       if (g.outer) "OUTER" else "",
       g.generator.sql,
-      SQLBuilder.newSubqueryName,
+      newSubqueryName(),
       "AS",
       columnAliases
     )
@@ -477,7 +480,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
     }
 
     private def addSubquery(plan: LogicalPlan): SubqueryAlias = {
-      SubqueryAlias(SQLBuilder.newSubqueryName, plan)
+      SubqueryAlias(newSubqueryName(), plan)
     }
 
     private def addSubqueryIfNeeded(plan: LogicalPlan): LogicalPlan = plan match {
@@ -514,9 +517,3 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends
Loggi
     }
   }
 }
-
-object SQLBuilder {
-  private val nextSubqueryId = new AtomicLong(0)
-
-  private def newSubqueryName: String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}"
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


Mime
View raw message