[SPARK-5213] [SQL] Remove the duplicated SparkSQLParser

This is a follow up of #5827 to remove the additional `SparkSQLParser`

Author: Cheng Hao <hao.cheng@intel.com>

Closes #5965 from chenghao-intel/remove_sparksqlparser and squashes the following commits:

509a233 [Cheng Hao] Remove the HiveQlQueryExecution
a5f9e3b [Cheng Hao] Remove the duplicated SparkSQLParser

(cherry picked from commit 074d75d4c8)
Signed-off-by: Michael Armbrust <michael@databricks.com>
This commit is contained in:
Cheng Hao 2015-05-07 12:09:54 -07:00 committed by Michael Armbrust
parent 86f141c90a
commit 2b0c423856
4 changed files with 11 additions and 23 deletions

View file

@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.lib.Node
import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.metadata.Table
import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.parse._
import org.apache.hadoop.hive.ql.plan.PlanUtils import org.apache.hadoop.hive.ql.plan.PlanUtils
import org.apache.spark.sql.{AnalysisException, SparkSQLParser} import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions._
@ -140,10 +140,7 @@ private[hive] object HiveQl {
"TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain.
) ++ nativeCommands ) ++ nativeCommands
protected val hqlParser = { protected val hqlParser = new ExtendedHiveQlParser
val fallback = new ExtendedHiveQlParser
new SparkSQLParser(fallback.parse(_))
}
/** /**
* A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations

View file

@ -94,7 +94,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME")
// Override so we can intercept relative paths and rewrite them to point at hive. // Override so we can intercept relative paths and rewrite them to point at hive.
override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) override def runSqlHive(sql: String): Seq[String] =
super.runSqlHive(rewritePaths(substitutor.substitute(this.hiveconf, sql)))
override def executePlan(plan: LogicalPlan): this.QueryExecution = override def executePlan(plan: LogicalPlan): this.QueryExecution =
new this.QueryExecution(plan) new this.QueryExecution(plan)
@ -157,22 +158,12 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
val describedTable = "DESCRIBE (\\w+)".r val describedTable = "DESCRIBE (\\w+)".r
val vs = new VariableSubstitution()
// we should substitute variables in hql to pass the text to parseSql() as a parameter.
// Hive parser need substituted text. HiveContext.sql() does this but return a DataFrame,
// while we need a logicalPlan so we cannot reuse that.
protected[hive] class HiveQLQueryExecution(hql: String)
extends this.QueryExecution(HiveQl.parseSql(vs.substitute(hiveconf, hql))) {
def hiveExec(): Seq[String] = runSqlHive(hql)
override def toString: String = hql + "\n" + super.toString
}
/** /**
* Override QueryExecution with special debug workflow. * Override QueryExecution with special debug workflow.
*/ */
class QueryExecution(logicalPlan: LogicalPlan) class QueryExecution(logicalPlan: LogicalPlan)
extends super.QueryExecution(logicalPlan) { extends super.QueryExecution(logicalPlan) {
def this(sql: String) = this(parseSql(sql))
override lazy val analyzed = { override lazy val analyzed = {
val describedTables = logical match { val describedTables = logical match {
case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil
@ -196,7 +187,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
protected[hive] implicit class SqlCmd(sql: String) { protected[hive] implicit class SqlCmd(sql: String) {
def cmd: () => Unit = { def cmd: () => Unit = {
() => new HiveQLQueryExecution(sql).stringResult(): Unit () => new QueryExecution(sql).stringResult(): Unit
} }
} }

View file

@ -129,7 +129,7 @@ abstract class HiveComparisonTest
} }
protected def prepareAnswer( protected def prepareAnswer(
hiveQuery: TestHive.type#HiveQLQueryExecution, hiveQuery: TestHive.type#QueryExecution,
answer: Seq[String]): Seq[String] = { answer: Seq[String]): Seq[String] = {
def isSorted(plan: LogicalPlan): Boolean = plan match { def isSorted(plan: LogicalPlan): Boolean = plan match {
@ -298,7 +298,7 @@ abstract class HiveComparisonTest
hiveCachedResults hiveCachedResults
} else { } else {
val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) val hiveQueries = queryList.map(new TestHive.QueryExecution(_))
// Make sure we can at least parse everything before attempting hive execution. // Make sure we can at least parse everything before attempting hive execution.
hiveQueries.foreach(_.logical) hiveQueries.foreach(_.logical)
val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map {
@ -346,7 +346,7 @@ abstract class HiveComparisonTest
// Run w/ catalyst // Run w/ catalyst
val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) =>
val query = new TestHive.HiveQLQueryExecution(queryString) val query = new TestHive.QueryExecution(queryString)
try { (query, prepareAnswer(query, query.stringResult())) } catch { try { (query, prepareAnswer(query, query.stringResult())) } catch {
case e: Throwable => case e: Throwable =>
val errorMessage = val errorMessage =
@ -402,7 +402,7 @@ abstract class HiveComparisonTest
// okay by running a simple query. If this fails then we halt testing since // okay by running a simple query. If this fails then we halt testing since
// something must have gone seriously wrong. // something must have gone seriously wrong.
try { try {
new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() new TestHive.QueryExecution("SELECT key FROM src").stringResult()
TestHive.runSqlHive("SELECT key FROM src") TestHive.runSqlHive("SELECT key FROM src")
} catch { } catch {
case e: Exception => case e: Exception =>

View file

@ -145,7 +145,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter {
expectedScannedColumns: Seq[String], expectedScannedColumns: Seq[String],
expectedPartValues: Seq[Seq[String]]): Unit = { expectedPartValues: Seq[Seq[String]]): Unit = {
test(s"$testCaseName - pruning test") { test(s"$testCaseName - pruning test") {
val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val plan = new TestHive.QueryExecution(sql).executedPlan
val actualOutputColumns = plan.output.map(_.name) val actualOutputColumns = plan.output.map(_.name)
val (actualScannedColumns, actualPartValues) = plan.collect { val (actualScannedColumns, actualPartValues) = plan.collect {
case p @ HiveTableScan(columns, relation, _) => case p @ HiveTableScan(columns, relation, _) =>