[SPARK-21307][REVERT][SQL] Remove SQLConf parameters from the parser-related classes

## What changes were proposed in this pull request?
Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first.

## How was this patch tested?
The existing test cases

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18568 from gatorsmile/revert18531.
This commit is contained in:
Xiao Li 2017-07-08 11:56:19 -07:00 committed by gatorsmile
parent 062c336d06
commit c3712b77a9
11 changed files with 115 additions and 109 deletions

View file

@ -74,7 +74,7 @@ class SessionCatalog(
functionRegistry, functionRegistry,
conf, conf,
new Configuration(), new Configuration(),
CatalystSqlParser, new CatalystSqlParser(conf),
DummyFunctionResourceLoader) DummyFunctionResourceLoader)
} }

View file

@ -45,9 +45,11 @@ import org.apache.spark.util.random.RandomSampler
* The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
* TableIdentifier. * TableIdentifier.
*/ */
class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging {
import ParserUtils._ import ParserUtils._
def this() = this(new SQLConf())
protected def typedVisit[T](ctx: ParseTree): T = { protected def typedVisit[T](ctx: ParseTree): T = {
ctx.accept(this).asInstanceOf[T] ctx.accept(this).asInstanceOf[T]
} }
@ -1457,7 +1459,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
* Special characters can be escaped by using Hive/C-style escaping. * Special characters can be escaped by using Hive/C-style escaping.
*/ */
private def createString(ctx: StringLiteralContext): String = { private def createString(ctx: StringLiteralContext): String = {
if (SQLConf.get.escapedStringLiterals) { if (conf.escapedStringLiterals) {
ctx.STRING().asScala.map(stringWithoutUnescape).mkString ctx.STRING().asScala.map(stringWithoutUnescape).mkString
} else { } else {
ctx.STRING().asScala.map(string).mkString ctx.STRING().asScala.map(string).mkString

View file

@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.catalyst.trees.Origin
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.types.{DataType, StructType}
/** /**
@ -121,8 +122,13 @@ abstract class AbstractSqlParser extends ParserInterface with Logging {
/** /**
* Concrete SQL parser for Catalyst-only SQL statements. * Concrete SQL parser for Catalyst-only SQL statements.
*/ */
class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser {
val astBuilder = new AstBuilder(conf)
}
/** For test-only. */
object CatalystSqlParser extends AbstractSqlParser { object CatalystSqlParser extends AbstractSqlParser {
val astBuilder = new AstBuilder val astBuilder = new AstBuilder(new SQLConf())
} }
/** /**

View file

@ -167,12 +167,12 @@ class ExpressionParserSuite extends PlanTest {
} }
test("like expressions with ESCAPED_STRING_LITERALS = true") { test("like expressions with ESCAPED_STRING_LITERALS = true") {
val parser = CatalystSqlParser val conf = new SQLConf()
withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> "true") { conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true")
assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", parser) val parser = new CatalystSqlParser(conf)
assertEqual("a rlike 'pattern\\\\'", 'a rlike "pattern\\\\", parser) assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", parser)
assertEqual("a rlike 'pattern\\t\\n'", 'a rlike "pattern\\t\\n", parser) assertEqual("a rlike 'pattern\\\\'", 'a rlike "pattern\\\\", parser)
} assertEqual("a rlike 'pattern\\t\\n'", 'a rlike "pattern\\t\\n", parser)
} }
test("is null expressions") { test("is null expressions") {
@ -435,85 +435,86 @@ class ExpressionParserSuite extends PlanTest {
} }
test("strings") { test("strings") {
val parser = CatalystSqlParser
Seq(true, false).foreach { escape => Seq(true, false).foreach { escape =>
withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> escape.toString) { val conf = new SQLConf()
// tests that have same result whatever the conf is conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, escape.toString)
// Single Strings. val parser = new CatalystSqlParser(conf)
assertEqual("\"hello\"", "hello", parser)
assertEqual("'hello'", "hello", parser)
// Multi-Strings. // tests that have same result whatever the conf is
assertEqual("\"hello\" 'world'", "helloworld", parser) // Single Strings.
assertEqual("'hello' \" \" 'world'", "hello world", parser) assertEqual("\"hello\"", "hello", parser)
assertEqual("'hello'", "hello", parser)
// 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a // Multi-Strings.
// regular '%'; to get the correct result you need to add another escaped '\'. assertEqual("\"hello\" 'world'", "helloworld", parser)
// TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method? assertEqual("'hello' \" \" 'world'", "hello world", parser)
assertEqual("'pattern%'", "pattern%", parser)
assertEqual("'no-pattern\\%'", "no-pattern\\%", parser)
// tests that have different result regarding the conf // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
if (escape) { // regular '%'; to get the correct result you need to add another escaped '\'.
// When SQLConf.ESCAPED_STRING_LITERALS is enabled, string literal parsing fallbacks to // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
// Spark 1.6 behavior. assertEqual("'pattern%'", "pattern%", parser)
assertEqual("'no-pattern\\%'", "no-pattern\\%", parser)
// 'LIKE' string literals. // tests that have different result regarding the conf
assertEqual("'pattern\\\\%'", "pattern\\\\%", parser) if (escape) {
assertEqual("'pattern\\\\\\%'", "pattern\\\\\\%", parser) // When SQLConf.ESCAPED_STRING_LITERALS is enabled, string literal parsing fallbacks to
// Spark 1.6 behavior.
// Escaped characters. // 'LIKE' string literals.
// Unescape string literal "'\\0'" for ASCII NUL (X'00') doesn't work assertEqual("'pattern\\\\%'", "pattern\\\\%", parser)
// when ESCAPED_STRING_LITERALS is enabled. assertEqual("'pattern\\\\\\%'", "pattern\\\\\\%", parser)
// It is parsed literally.
assertEqual("'\\0'", "\\0", parser)
// Note: Single quote follows 1.6 parsing behavior when ESCAPED_STRING_LITERALS is // Escaped characters.
// enabled. // Unescape string literal "'\\0'" for ASCII NUL (X'00') doesn't work
val e = intercept[ParseException](parser.parseExpression("'\''")) // when ESCAPED_STRING_LITERALS is enabled.
assert(e.message.contains("extraneous input '''")) // It is parsed literally.
assertEqual("'\\0'", "\\0", parser)
// The unescape special characters (e.g., "\\t") for 2.0+ don't work // Note: Single quote follows 1.6 parsing behavior when ESCAPED_STRING_LITERALS is enabled.
// when ESCAPED_STRING_LITERALS is enabled. They are parsed literally. val e = intercept[ParseException](parser.parseExpression("'\''"))
assertEqual("'\\\"'", "\\\"", parser) // Double quote assert(e.message.contains("extraneous input '''"))
assertEqual("'\\b'", "\\b", parser) // Backspace
assertEqual("'\\n'", "\\n", parser) // Newline
assertEqual("'\\r'", "\\r", parser) // Carriage return
assertEqual("'\\t'", "\\t", parser) // Tab character
// The unescape Octals for 2.0+ don't work when ESCAPED_STRING_LITERALS is enabled. // The unescape special characters (e.g., "\\t") for 2.0+ don't work
// They are parsed literally. // when ESCAPED_STRING_LITERALS is enabled. They are parsed literally.
assertEqual("'\\110\\145\\154\\154\\157\\041'", "\\110\\145\\154\\154\\157\\041", parser) assertEqual("'\\\"'", "\\\"", parser) // Double quote
// The unescape Unicode for 2.0+ doesn't work when ESCAPED_STRING_LITERALS is enabled. assertEqual("'\\b'", "\\b", parser) // Backspace
// They are parsed literally. assertEqual("'\\n'", "\\n", parser) // Newline
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", assertEqual("'\\r'", "\\r", parser) // Carriage return
"\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029", parser) assertEqual("'\\t'", "\\t", parser) // Tab character
} else {
// Default behavior
// 'LIKE' string literals. // The unescape Octals for 2.0+ don't work when ESCAPED_STRING_LITERALS is enabled.
assertEqual("'pattern\\\\%'", "pattern\\%", parser) // They are parsed literally.
assertEqual("'pattern\\\\\\%'", "pattern\\\\%", parser) assertEqual("'\\110\\145\\154\\154\\157\\041'", "\\110\\145\\154\\154\\157\\041", parser)
// The unescape Unicode for 2.0+ doesn't work when ESCAPED_STRING_LITERALS is enabled.
// They are parsed literally.
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'",
"\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029", parser)
} else {
// Default behavior
// Escaped characters. // 'LIKE' string literals.
// See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html assertEqual("'pattern\\\\%'", "pattern\\%", parser)
assertEqual("'\\0'", "\u0000", parser) // ASCII NUL (X'00') assertEqual("'pattern\\\\\\%'", "pattern\\\\%", parser)
assertEqual("'\\''", "\'", parser) // Single quote
assertEqual("'\\\"'", "\"", parser) // Double quote
assertEqual("'\\b'", "\b", parser) // Backspace
assertEqual("'\\n'", "\n", parser) // Newline
assertEqual("'\\r'", "\r", parser) // Carriage return
assertEqual("'\\t'", "\t", parser) // Tab character
assertEqual("'\\Z'", "\u001A", parser) // ASCII 26 - CTRL + Z (EOF on windows)
// Octals // Escaped characters.
assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!", parser) // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
assertEqual("'\\0'", "\u0000", parser) // ASCII NUL (X'00')
assertEqual("'\\''", "\'", parser) // Single quote
assertEqual("'\\\"'", "\"", parser) // Double quote
assertEqual("'\\b'", "\b", parser) // Backspace
assertEqual("'\\n'", "\n", parser) // Newline
assertEqual("'\\r'", "\r", parser) // Carriage return
assertEqual("'\\t'", "\t", parser) // Tab character
assertEqual("'\\Z'", "\u001A", parser) // ASCII 26 - CTRL + Z (EOF on windows)
// Unicode // Octals
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)", assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!", parser)
parser)
} // Unicode
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)",
parser)
} }
} }
} }

View file

@ -39,11 +39,10 @@ import org.apache.spark.sql.types.StructType
/** /**
* Concrete parser for Spark SQL statements. * Concrete parser for Spark SQL statements.
*/ */
class SparkSqlParser extends AbstractSqlParser { class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser {
val astBuilder = new SparkSqlAstBuilder(conf)
val astBuilder = new SparkSqlAstBuilder private val substitutor = new VariableSubstitution(conf)
private val substitutor = new VariableSubstitution
protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
super.parse(substitutor.substitute(command))(toResult) super.parse(substitutor.substitute(command))(toResult)
@ -53,11 +52,9 @@ class SparkSqlParser extends AbstractSqlParser {
/** /**
* Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
*/ */
class SparkSqlAstBuilder extends AstBuilder { class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
import org.apache.spark.sql.catalyst.parser.ParserUtils._ import org.apache.spark.sql.catalyst.parser.ParserUtils._
private def conf: SQLConf = SQLConf.get
/** /**
* Create a [[SetCommand]] logical plan. * Create a [[SetCommand]] logical plan.
* *

View file

@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint} import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint}
import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.SparkSqlParser
import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.expressions.UserDefinedFunction
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.util.Utils import org.apache.spark.util.Utils
@ -1275,7 +1276,7 @@ object functions {
*/ */
def expr(expr: String): Column = { def expr(expr: String): Column = {
val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse { val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse {
new SparkSqlParser new SparkSqlParser(new SQLConf)
} }
Column(parser.parseExpression(expr)) Column(parser.parseExpression(expr))
} }

View file

@ -114,7 +114,7 @@ abstract class BaseSessionStateBuilder(
* Note: this depends on the `conf` field. * Note: this depends on the `conf` field.
*/ */
protected lazy val sqlParser: ParserInterface = { protected lazy val sqlParser: ParserInterface = {
extensions.buildParser(session, new SparkSqlParser) extensions.buildParser(session, new SparkSqlParser(conf))
} }
/** /**

View file

@ -25,9 +25,7 @@ import org.apache.spark.internal.config._
* *
* Variable substitution is controlled by `SQLConf.variableSubstituteEnabled`. * Variable substitution is controlled by `SQLConf.variableSubstituteEnabled`.
*/ */
class VariableSubstitution { class VariableSubstitution(conf: SQLConf) {
private def conf = SQLConf.get
private val provider = new ConfigProvider { private val provider = new ConfigProvider {
override def get(key: String): Option[String] = Option(conf.getConfString(key, "")) override def get(key: String): Option[String] = Option(conf.getConfString(key, ""))

View file

@ -37,7 +37,8 @@ import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType
*/ */
class SparkSqlParserSuite extends AnalysisTest { class SparkSqlParserSuite extends AnalysisTest {
private lazy val parser = new SparkSqlParser val newConf = new SQLConf
private lazy val parser = new SparkSqlParser(newConf)
/** /**
* Normalizes plans: * Normalizes plans:
@ -284,7 +285,6 @@ class SparkSqlParserSuite extends AnalysisTest {
} }
test("query organization") { test("query organization") {
val conf = SQLConf.get
// Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
val baseSql = "select * from t" val baseSql = "select * from t"
val basePlan = val basePlan =
@ -293,20 +293,20 @@ class SparkSqlParserSuite extends AnalysisTest {
assertEqual(s"$baseSql distribute by a, b", assertEqual(s"$baseSql distribute by a, b",
RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil,
basePlan, basePlan,
numPartitions = conf.numShufflePartitions)) numPartitions = newConf.numShufflePartitions))
assertEqual(s"$baseSql distribute by a sort by b", assertEqual(s"$baseSql distribute by a sort by b",
Sort(SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, Sort(SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil,
global = false, global = false,
RepartitionByExpression(UnresolvedAttribute("a") :: Nil, RepartitionByExpression(UnresolvedAttribute("a") :: Nil,
basePlan, basePlan,
numPartitions = conf.numShufflePartitions))) numPartitions = newConf.numShufflePartitions)))
assertEqual(s"$baseSql cluster by a, b", assertEqual(s"$baseSql cluster by a, b",
Sort(SortOrder(UnresolvedAttribute("a"), Ascending) :: Sort(SortOrder(UnresolvedAttribute("a"), Ascending) ::
SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil,
global = false, global = false,
RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil,
basePlan, basePlan,
numPartitions = conf.numShufflePartitions))) numPartitions = newConf.numShufflePartitions)))
} }
test("pipeline concatenation") { test("pipeline concatenation") {

View file

@ -29,13 +29,13 @@ import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.plans.logical.Project
import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.SparkSqlParser
import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.datasources.CreateTable
import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
// TODO: merge this with DDLSuite (SPARK-14441) // TODO: merge this with DDLSuite (SPARK-14441)
class DDLCommandSuite extends PlanTest { class DDLCommandSuite extends PlanTest {
private lazy val parser = new SparkSqlParser private lazy val parser = new SparkSqlParser(new SQLConf)
private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = {
val e = intercept[ParseException] { val e = intercept[ParseException] {

View file

@ -18,11 +18,12 @@
package org.apache.spark.sql.internal package org.apache.spark.sql.internal
import org.apache.spark.SparkFunSuite import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.AnalysisException
class VariableSubstitutionSuite extends SparkFunSuite with PlanTest { class VariableSubstitutionSuite extends SparkFunSuite {
private lazy val sub = new VariableSubstitution private lazy val conf = new SQLConf
private lazy val sub = new VariableSubstitution(conf)
test("system property") { test("system property") {
System.setProperty("varSubSuite.var", "abcd") System.setProperty("varSubSuite.var", "abcd")
@ -34,26 +35,26 @@ class VariableSubstitutionSuite extends SparkFunSuite with PlanTest {
} }
test("Spark configuration variable") { test("Spark configuration variable") {
withSQLConf("some-random-string-abcd" -> "1234abcd") { conf.setConfString("some-random-string-abcd", "1234abcd")
assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd") assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd")
assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd") assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd")
assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd") assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd")
assert(sub.substitute("${some-random-string-abcd}") == "1234abcd") assert(sub.substitute("${some-random-string-abcd}") == "1234abcd")
}
} }
test("multiple substitutes") { test("multiple substitutes") {
val q = "select ${bar} ${foo} ${doo} this is great" val q = "select ${bar} ${foo} ${doo} this is great"
withSQLConf("bar" -> "1", "foo" -> "2", "doo" -> "3") { conf.setConfString("bar", "1")
assert(sub.substitute(q) == "select 1 2 3 this is great") conf.setConfString("foo", "2")
} conf.setConfString("doo", "3")
assert(sub.substitute(q) == "select 1 2 3 this is great")
} }
test("test nested substitutes") { test("test nested substitutes") {
val q = "select ${bar} ${foo} this is great" val q = "select ${bar} ${foo} this is great"
withSQLConf("bar" -> "1", "foo" -> "${bar}") { conf.setConfString("bar", "1")
assert(sub.substitute(q) == "select 1 1 this is great") conf.setConfString("foo", "${bar}")
} assert(sub.substitute(q) == "select 1 1 this is great")
} }
} }