[SPARK-14841][SQL] Move SQLBuilder into sql/core
## What changes were proposed in this pull request? This patch moves SQLBuilder into sql/core so we can in the future move view generation also into sql/core. ## How was this patch tested? Also moved unit tests. Author: Reynold Xin <rxin@databricks.com> Author: Wenchen Fan <wenchen@databricks.com> Closes #12602 from rxin/SPARK-14841.
This commit is contained in:
parent
8098f15857
commit
aeb52bea56
|
@ -15,15 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hive
|
||||
package org.apache.spark.sql.catalyst
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{Dataset, SQLContext}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.Dataset
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogRelation
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.optimizer.{CollapseProject, CombineUnions}
|
||||
|
@ -39,10 +38,10 @@ import org.apache.spark.sql.types.{ByteType, DataType, IntegerType, NullType}
|
|||
* representations (e.g. logical plans that operate on local Scala collections), or are simply not
|
||||
* supported by this builder (yet).
|
||||
*/
|
||||
class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Logging {
|
||||
class SQLBuilder(logicalPlan: LogicalPlan) extends Logging {
|
||||
require(logicalPlan.resolved, "SQLBuilder only supports resolved logical query plans")
|
||||
|
||||
def this(df: Dataset[_]) = this(df.queryExecution.analyzed, df.sqlContext)
|
||||
def this(df: Dataset[_]) = this(df.queryExecution.analyzed)
|
||||
|
||||
private val nextSubqueryId = new AtomicLong(0)
|
||||
private def newSubqueryName(): String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}"
|
||||
|
@ -70,7 +69,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi
|
|||
try {
|
||||
val replaced = finalPlan.transformAllExpressions {
|
||||
case e: SubqueryExpression =>
|
||||
SubqueryHolder(new SQLBuilder(e.query, sqlContext).toSQL)
|
||||
SubqueryHolder(new SQLBuilder(e.query).toSQL)
|
||||
case e: NonSQLExpression =>
|
||||
throw new UnsupportedOperationException(
|
||||
s"Expression $e doesn't have a SQL representation"
|
||||
|
@ -292,10 +291,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi
|
|||
a.groupingExpressions.map(_.asInstanceOf[Attribute]))
|
||||
}
|
||||
|
||||
private def groupingSetToSQL(
|
||||
agg: Aggregate,
|
||||
expand: Expand,
|
||||
project: Project): String = {
|
||||
private def groupingSetToSQL(agg: Aggregate, expand: Expand, project: Project): String = {
|
||||
assert(agg.groupingExpressions.length > 1)
|
||||
|
||||
// The last column of Expand is always grouping ID
|
|
@ -212,6 +212,7 @@ abstract class QueryTest extends PlanTest {
|
|||
case _: ObjectProducer => return
|
||||
case _: AppendColumns => return
|
||||
case _: LogicalRelation => return
|
||||
case p if p.getClass.getSimpleName == "MetastoreRelation" => return
|
||||
case _: MemoryPlan => return
|
||||
}.transformAllExpressions {
|
||||
case a: ImperativeAggregate => return
|
||||
|
|
|
@ -20,11 +20,12 @@ package org.apache.spark.sql.hive.execution
|
|||
import scala.util.control.NonFatal
|
||||
|
||||
import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
|
||||
import org.apache.spark.sql.catalyst.SQLBuilder
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable}
|
||||
import org.apache.spark.sql.catalyst.expressions.Alias
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveSessionState, SQLBuilder}
|
||||
import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveSessionState}
|
||||
|
||||
/**
|
||||
* Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of
|
||||
|
@ -128,7 +129,7 @@ private[hive] case class CreateViewAsSelect(
|
|||
}
|
||||
sqlContext.executePlan(Project(projectList, child)).analyzed
|
||||
}
|
||||
new SQLBuilder(logicalPlan, sqlContext).toSQL
|
||||
new SQLBuilder(logicalPlan).toSQL
|
||||
}
|
||||
|
||||
// escape backtick with double-backtick in column name and wrap it with backtick.
|
||||
|
|
|
@ -15,13 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hive
|
||||
package org.apache.spark.sql.catalyst
|
||||
|
||||
import java.sql.Timestamp
|
||||
|
||||
import org.apache.spark.sql.catalyst.dsl.expressions._
|
||||
import org.apache.spark.sql.catalyst.expressions.{If, Literal}
|
||||
|
||||
|
||||
class ExpressionSQLBuilderSuite extends SQLBuilderTest {
|
||||
test("literal") {
|
||||
checkSQL(Literal("foo"), "\"foo\"")
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hive
|
||||
package org.apache.spark.sql.catalyst
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hive
|
||||
package org.apache.spark.sql.catalyst
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hive
|
||||
package org.apache.spark.sql.catalyst
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
|
@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression
|
|||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.hive.test.TestHiveSingleton
|
||||
|
||||
|
||||
abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
|
||||
protected def checkSQL(e: Expression, expectedSQL: String): Unit = {
|
||||
val actualSQL = e.sql
|
||||
|
@ -42,7 +43,7 @@ abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
|
|||
}
|
||||
|
||||
protected def checkSQL(plan: LogicalPlan, expectedSQL: String): Unit = {
|
||||
val generatedSQL = try new SQLBuilder(plan, hiveContext).toSQL catch { case NonFatal(e) =>
|
||||
val generatedSQL = try new SQLBuilder(plan).toSQL catch { case NonFatal(e) =>
|
||||
fail(
|
||||
s"""Cannot convert the following logical query plan to SQL:
|
||||
|
|
|
@ -25,12 +25,12 @@ import scala.util.control.NonFatal
|
|||
import org.scalatest.{BeforeAndAfterAll, GivenWhenThen}
|
||||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.catalyst.SQLBuilder
|
||||
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.catalyst.util._
|
||||
import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExplainCommand, HiveNativeCommand, SetCommand}
|
||||
import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable}
|
||||
import org.apache.spark.sql.hive.SQLBuilder
|
||||
import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution}
|
||||
|
||||
/**
|
||||
|
@ -407,7 +407,7 @@ abstract class HiveComparisonTest
|
|||
originalQuery
|
||||
} else {
|
||||
val convertedSQL = try {
|
||||
new SQLBuilder(originalQuery.analyzed, TestHive).toSQL
|
||||
new SQLBuilder(originalQuery.analyzed).toSQL
|
||||
} catch {
|
||||
case NonFatal(e) => fail(
|
||||
s"""Cannot convert the following HiveQL query plan back to SQL query string:
|
||||
|
|
Loading…
Reference in a new issue