Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ class SessionCatalog(
functionRegistry,
conf,
new Configuration(),
CatalystSqlParser,
new CatalystSqlParser(conf),
DummyFunctionResourceLoader)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
import org.apache.spark.util.random.RandomSampler
Expand All @@ -44,9 +45,11 @@ import org.apache.spark.util.random.RandomSampler
* The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
* TableIdentifier.
*/
class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging {
import ParserUtils._

def this() = this(new SQLConf())

protected def typedVisit[T](ctx: ParseTree): T = {
ctx.accept(this).asInstanceOf[T]
}
Expand Down Expand Up @@ -1406,7 +1409,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
* Special characters can be escaped by using Hive/C-style escaping.
*/
private def createString(ctx: StringLiteralContext): String = {
ctx.STRING().asScala.map(string).mkString
if (conf.escapedStringLiterals) {
ctx.STRING().asScala.map(stringWithoutUnescape).mkString
} else {
ctx.STRING().asScala.map(string).mkString
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.Origin
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, StructType}

/**
Expand Down Expand Up @@ -120,8 +121,13 @@ abstract class AbstractSqlParser extends ParserInterface with Logging {
/**
* 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 {
val astBuilder = new AstBuilder
val astBuilder = new AstBuilder(new SQLConf())
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,12 @@ object ParserUtils {
/** Convert a string node into a string. */
def string(node: TerminalNode): String = unescapeSQLString(node.getText)

/** Convert a string node into a string without unescaping. */
def stringWithoutUnescape(node: TerminalNode): String = {
// STRING parser rule forces that the input always has quotes at the starting and ending.
node.getText.slice(1, node.getText.size - 1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For safety, do we still need to check whether the starting and ending characters are quotes?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The string rule in SqlBase.g4 forces that the input has always quotes at the starting and ending. I may add a comment here.

}

/** Get the origin (line and position) of the token. */
def position(token: Token): Origin = {
val opt = Option(token)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,14 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals")
.internal()
.doc("When true, string literals (including regex patterns) remains escaped in our SQL " +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: remains -> remain

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure.

"parser. The default is false since Spark 2.0. Setting it to true can restore the behavior " +
"prior to Spark 2.0.")
.booleanConf
.createWithDefault(false)

val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema")
.doc("When true, the Parquet data source merges schemas collected from all data files, " +
"otherwise the schema is picked from the summary file or a random data file " +
Expand Down Expand Up @@ -911,6 +919,8 @@ class SQLConf extends Serializable with Logging {

def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED)

def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS)

/**
* Returns the [[Resolver]] for the current configuration, which can be used to determine if two
* identifiers are equal.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval

Expand All @@ -39,12 +40,17 @@ class ExpressionParserSuite extends PlanTest {
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._

def assertEqual(sqlCommand: String, e: Expression): Unit = {
compareExpressions(parseExpression(sqlCommand), e)
val defaultParser = CatalystSqlParser

def assertEqual(
sqlCommand: String,
e: Expression,
parser: ParserInterface = defaultParser): Unit = {
compareExpressions(parser.parseExpression(sqlCommand), e)
}

def intercept(sqlCommand: String, messages: String*): Unit = {
val e = intercept[ParseException](parseExpression(sqlCommand))
val e = intercept[ParseException](defaultParser.parseExpression(sqlCommand))
messages.foreach { message =>
assert(e.message.contains(message))
}
Expand Down Expand Up @@ -101,7 +107,7 @@ class ExpressionParserSuite extends PlanTest {
test("long binary logical expressions") {
def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
val e = parseExpression(sql)
val e = defaultParser.parseExpression(sql)
assert(e.collect { case _: EqualTo => true }.size === 1000)
assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
}
Expand Down Expand Up @@ -160,6 +166,15 @@ class ExpressionParserSuite extends PlanTest {
assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
}

test("like expressions with ESCAPED_STRING_LITERALS = true") {
val conf = new SQLConf()
conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true")
val parser = new CatalystSqlParser(conf)
assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", 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") {
assertEqual("a is null", 'a.isNull)
assertEqual("a is not null", 'a.isNotNull)
Expand Down Expand Up @@ -413,38 +428,102 @@ class ExpressionParserSuite extends PlanTest {
}

test("strings") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about something like

Seq(true, false).foreach { escape =>
  val conf = new SQLConf()
  conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true")
  val parser = new CatalystSqlParser(conf)

  // tests that have same result whatever the conf is
  assertEqual("\"hello\"", "hello")
  ...

  // tests that have different result regarding the conf
  if (escape) {
    assert(...) 
    ...
  } else {
    assert(...)
    ...
  }

}

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure.

// Single Strings.
assertEqual("\"hello\"", "hello")
assertEqual("'hello'", "hello")

// Multi-Strings.
assertEqual("\"hello\" 'world'", "helloworld")
assertEqual("'hello' \" \" 'world'", "hello world")

// 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
// regular '%'; to get the correct result you need to add another escaped '\'.
// TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
assertEqual("'pattern%'", "pattern%")
assertEqual("'no-pattern\\%'", "no-pattern\\%")
assertEqual("'pattern\\\\%'", "pattern\\%")
assertEqual("'pattern\\\\\\%'", "pattern\\\\%")

// Escaped characters.
// See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
assertEqual("'\\0'", "\u0000") // ASCII NUL (X'00')
assertEqual("'\\''", "\'") // Single quote
assertEqual("'\\\"'", "\"") // Double quote
assertEqual("'\\b'", "\b") // Backspace
assertEqual("'\\n'", "\n") // Newline
assertEqual("'\\r'", "\r") // Carriage return
assertEqual("'\\t'", "\t") // Tab character
assertEqual("'\\Z'", "\u001A") // ASCII 26 - CTRL + Z (EOF on windows)

// Octals
assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!")

// Unicode
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)")
// The SQL commands when ESCAPED_STRING_LITERALS = false (default behavior)
val sqlCommands = Seq(
// Single Strings.
"\"hello\"",
"'hello'",
// Multi-Strings.
"\"hello\" 'world'",
"'hello' \" \" 'world'",
// 'LIKE' string literals.
"'pattern%'",
"'no-pattern\\%'",
"'pattern\\\\%'",
"'pattern\\\\\\%'",
// Escaped characters.
"'\\0'",
"'\\\"'",
"'\\b'",
"'\\n'",
"'\\r'",
"'\\t'",
// Octals
"'\\110\\145\\154\\154\\157\\041'",
// Unicode
"'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'")

// The SQL commands when ESCAPED_STRING_LITERALS = true
val fallbackSqlCommands = Seq(
// Single Strings.
"\"hello\"",
"'hello'",
// Multi-Strings.
"\"hello\" 'world'",
"'hello' \" \" 'world'",
// 'LIKE' string literals.
"'pattern%'",
"'no-pattern\\%'",
"'pattern\\%'",
"'pattern\\\\%'",
// Escaped characters.
"'\0'",
"'\"'",
"'\b'",
"'\n'",
"'\r'",
"'\t'",
// Octals
"'\110\145\154\154\157\041'",
// Unicode
"'\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029'")

val expectedResults = Seq(
// Single Strings.
"hello",
"hello",
// Multi-Strings.
"helloworld",
"hello world",
// 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
// regular '%'; to get the correct result you need to add another escaped '\'.
// TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
"pattern%",
"no-pattern\\%",
"pattern\\%",
"pattern\\\\%",
// Escaped characters.
// See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
"\u0000", // ASCII NUL (X'00')
"\"", // Double quote
"\b", // Backspace
"\n", // Newline
"\r", // Carriage return
"\t", // Tab character
// Octals
"Hello!",
// Unicode
"World :)")

val tests = Seq(("false", sqlCommands), ("true", fallbackSqlCommands))

tests.map { case (escapedStringLiterals, commands) =>
val conf = new SQLConf()
conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, escapedStringLiterals)
val parser = new CatalystSqlParser(conf)
commands.zip(expectedResults).foreach { case (sqlCommand, expected) =>
assertEqual(sqlCommand, expected, parser)
}
if (escapedStringLiterals == "false") {
assertEqual("'\\''", "\'", parser) // Single quote
assertEqual("'\\Z'", "\u001A", parser) // ASCII 26 - CTRL + Z (EOF on windows)
} else {
// Note: Single quote follows 1.6 parsing behavior when ESCAPED_STRING_LITERALS is enabled.
val e = intercept[ParseException](parser.parseExpression("'\''"))
assert(e.message.contains("extraneous input '''"))
}

}
}

test("intervals") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser {
/**
* Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
*/
class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
import org.apache.spark.sql.catalyst.parser.ParserUtils._

/**
Expand Down
13 changes: 13 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SortExec}
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchange}
import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._

Expand Down Expand Up @@ -1168,6 +1169,18 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
val ds = Seq(WithMapInOption(Some(Map(1 -> 1)))).toDS()
checkDataset(ds, WithMapInOption(Some(Map(1 -> 1))))
}

test("do not unescaped regex pattern string") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add jira id and when we should not unescape

withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> "true") {
val data = Seq("\u0020\u0021\u0023", "abc")
val df = data.toDF()
val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))
val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")
checkAnswer(rlike1, rlike2)
assert(rlike3.count() == 0)
}
}
}

case class WithImmutableMap(id: String, map_test: scala.collection.immutable.Map[Long, String])
Expand Down