aboutsummaryrefslogtreecommitdiff
path: root/sql/core
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2016-01-18 13:55:42 -0800
committerReynold Xin <rxin@databricks.com>2016-01-18 13:55:42 -0800
commit38c3c0e31a00aed56f0bc0791a2789c845a3fd61 (patch)
treeb6a220eb6731017b54b53a24a0d42b80b2a71c54 /sql/core
parentb8cb548a4394221f2b029c84c7f130774da69e3a (diff)
downloadspark-38c3c0e31a00aed56f0bc0791a2789c845a3fd61.tar.gz
spark-38c3c0e31a00aed56f0bc0791a2789c845a3fd61.tar.bz2
spark-38c3c0e31a00aed56f0bc0791a2789c845a3fd61.zip
[SPARK-12855][SQL] Remove parser dialect developer API
This pull request removes the public developer parser API for external parsers. Given everything a parser depends on (e.g. logical plans and expressions) are internal and not stable, external parsers will break with every release of Spark. It is a bad idea to create the illusion that Spark actually supports pluggable parsers. In addition, this also reduces incentives for 3rd party projects to contribute parse improvements back to Spark. Author: Reynold Xin <rxin@databricks.com> Closes #10801 from rxin/SPARK-12855.
Diffstat (limited to 'sql/core')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala20
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala35
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/functions.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala20
6 files changed, 8 insertions, 82 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
index 4c1eb0b30b..2d664d3ee6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
@@ -278,11 +278,6 @@ private[spark] object SQLConf {
doc = "When true, common subexpressions will be eliminated.",
isPublic = false)
- val DIALECT = stringConf(
- "spark.sql.dialect",
- defaultValue = Some("sql"),
- doc = "The default SQL dialect to use.")
-
val CASE_SENSITIVE = booleanConf("spark.sql.caseSensitive",
defaultValue = Some(true),
doc = "Whether the query analyzer should be case sensitive or not.")
@@ -524,21 +519,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with ParserCon
new java.util.HashMap[String, String]())
/** ************************ Spark SQL Params/Hints ******************* */
- // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext?
-
- /**
- * The SQL dialect that is used when parsing queries. This defaults to 'sql' which uses
- * a simple SQL parser provided by Spark SQL. This is currently the only option for users of
- * SQLContext.
- *
- * When using a HiveContext, this value defaults to 'hiveql', which uses the Hive 0.12.0 HiveQL
- * parser. Users can change this to 'sql' if they want to run queries that aren't supported by
- * HiveQL (e.g., SELECT 1).
- *
- * Note that the choice of dialect does not affect things like what tables are available or
- * how query execution is performed.
- */
- private[spark] def dialect: String = getConf(DIALECT)
private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 18ddffe1be..b8c8c78b91 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicReference
import scala.collection.JavaConverters._
import scala.collection.immutable
import scala.reflect.runtime.universe.TypeTag
-import scala.util.control.NonFatal
import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.annotation.{DeveloperApi, Experimental}
@@ -33,13 +32,11 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
import org.apache.spark.sql.{execution => sparkexecution}
import org.apache.spark.sql.SQLConf.SQLConfEntry
-import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, _}
+import org.apache.spark.sql.catalyst.{InternalRow, _}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.encoders.encoderFor
-import org.apache.spark.sql.catalyst.errors.DialectException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
-import org.apache.spark.sql.catalyst.parser.ParserConf
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution._
@@ -206,30 +203,10 @@ class SQLContext private[sql](
protected[sql] lazy val optimizer: Optimizer = new SparkOptimizer(this)
@transient
- protected[sql] val ddlParser = new DDLParser(sqlParser)
+ protected[sql] val sqlParser: ParserDialect = new SparkSQLParser(new SparkQl(conf))
@transient
- protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect())
-
- protected[sql] def getSQLDialect(): ParserDialect = {
- try {
- val clazz = Utils.classForName(dialectClassName)
- clazz.getConstructor(classOf[ParserConf])
- .newInstance(conf)
- .asInstanceOf[ParserDialect]
- } catch {
- case NonFatal(e) =>
- // Since we didn't find the available SQL Dialect, it will fail even for SET command:
- // SET spark.sql.dialect=sql; Let's reset as default dialect automatically.
- val dialect = conf.dialect
- // reset the sql dialect
- conf.unsetConf(SQLConf.DIALECT)
- // throw out the exception, and the default sql dialect will take effect for next query.
- throw new DialectException(
- s"""Instantiating dialect '$dialect' failed.
- |Reverting to default dialect '${conf.dialect}'""".stripMargin, e)
- }
- }
+ protected[sql] val ddlParser: DDLParser = new DDLParser(sqlParser)
protected[sql] def parseSql(sql: String): LogicalPlan = ddlParser.parse(sql, false)
@@ -239,12 +216,6 @@ class SQLContext private[sql](
protected[sql] def executePlan(plan: LogicalPlan) =
new sparkexecution.QueryExecution(this, plan)
- protected[sql] def dialectClassName = if (conf.dialect == "sql") {
- classOf[SparkQl].getCanonicalName
- } else {
- conf.dialect
- }
-
/**
* Add a jar to SQLContext
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 2e2ce88211..3cfa3dfd9c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -201,13 +201,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm
case Some((key, None)) =>
val runFunc = (sqlContext: SQLContext) => {
val value =
- try {
- if (key == SQLConf.DIALECT.key) {
- sqlContext.conf.dialect
- } else {
- sqlContext.getConf(key)
- }
- } catch {
+ try sqlContext.getConf(key) catch {
case _: NoSuchElementException => "<undefined>"
}
Seq(Row(key, value))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
index 10655a85cc..4dea947f6a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
@@ -37,9 +37,10 @@ class DDLParser(fallback: => ParserDialect)
override def parseExpression(sql: String): Expression = fallback.parseExpression(sql)
- override def parseTableIdentifier(sql: String): TableIdentifier =
-
+ override def parseTableIdentifier(sql: String): TableIdentifier = {
fallback.parseTableIdentifier(sql)
+ }
+
def parse(input: String, exceptionOnError: Boolean): LogicalPlan = {
try {
parsePlan(input)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 8c2530fd68..3a27466176 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -1064,7 +1064,7 @@ object functions extends LegacyFunctions {
* @group normal_funcs
*/
def expr(expr: String): Column = {
- val parser = SQLContext.getActive().map(_.getSQLDialect()).getOrElse(new CatalystQl())
+ val parser = SQLContext.getActive().map(_.sqlParser).getOrElse(new CatalystQl())
Column(parser.parseExpression(expr))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index bdb9421cc1..d7f182352b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -23,7 +23,6 @@ import java.sql.Timestamp
import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.catalyst.CatalystQl
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
-import org.apache.spark.sql.catalyst.errors.DialectException
import org.apache.spark.sql.catalyst.parser.ParserConf
import org.apache.spark.sql.execution.{aggregate, SparkQl}
import org.apache.spark.sql.execution.joins.{CartesianProduct, SortMergeJoin}
@@ -32,8 +31,6 @@ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._
-/** A SQL Dialect for testing purpose, and it can not be nested type */
-class MyDialect(conf: ParserConf) extends CatalystQl(conf)
class SQLQuerySuite extends QueryTest with SharedSQLContext {
import testImplicits._
@@ -148,23 +145,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
.count(), Row(24, 1) :: Row(14, 1) :: Nil)
}
- test("SQL Dialect Switching to a new SQL parser") {
- val newContext = new SQLContext(sparkContext)
- newContext.setConf("spark.sql.dialect", classOf[MyDialect].getCanonicalName())
- assert(newContext.getSQLDialect().getClass === classOf[MyDialect])
- assert(newContext.sql("SELECT 1").collect() === Array(Row(1)))
- }
-
- test("SQL Dialect Switch to an invalid parser with alias") {
- val newContext = new SQLContext(sparkContext)
- newContext.sql("SET spark.sql.dialect=MyTestClass")
- intercept[DialectException] {
- newContext.sql("SELECT 1")
- }
- // test if the dialect set back to DefaultSQLDialect
- assert(newContext.getSQLDialect().getClass === classOf[SparkQl])
- }
-
test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") {
checkAnswer(
sql("SELECT a FROM testData2 SORT BY a"),