Skip to content

Commit

Permalink
[SPARK-20399][SQL] Add a config to fallback string literal parsing co…
Browse files Browse the repository at this point in the history
…nsistent with old sql parser behavior

## What changes were proposed in this pull request?

The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string.

The following codes can reproduce it:

    val data = Seq("\u0020\u0021\u0023", "abc")
    val df = data.toDF()

    // 1st usage: works in 1.6
    // Let parser parse pattern string
    val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
    // 2nd usage: works in 1.6, 2.x
    // Call Column.rlike so the pattern string is a literal which doesn't go through parser
    val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))

    // In 2.x, we need add backslashes to make regex pattern parsed correctly
    val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")

Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #17887 from viirya/add-config-fallback-string-parsing.
  • Loading branch information
viirya authored and cloud-fan committed May 12, 2017
1 parent 04901dd commit 609ba5f
Show file tree
Hide file tree
Showing 9 changed files with 171 additions and 42 deletions.
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 @@ -86,6 +86,13 @@ abstract class StringRegexExpression extends BinaryExpression
escape character, the following character is matched literally. It is invalid to escape
any other character.
Since Spark 2.0, string literals are unescaped in our SQL parser. For example, in order
to match "\abc", the pattern should be "\\abc".
When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, it fallbacks
to Spark 1.6 behavior regarding string literal parsing. For example, if the config is
enabled, the pattern to match "\abc" should be "\abc".
Examples:
> SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%'
true
Expand Down Expand Up @@ -144,7 +151,31 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi
}

@ExpressionDescription(
usage = "str _FUNC_ regexp - Returns true if `str` matches `regexp`, or false otherwise.")
usage = "str _FUNC_ regexp - Returns true if `str` matches `regexp`, or false otherwise.",
extended = """
Arguments:
str - a string expression
regexp - a string expression. The pattern string should be a Java regular expression.
Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL parser.
For example, to match "\abc", a regular expression for `regexp` can be "^\\abc$".
There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to fallback
to the Spark 1.6 behavior regarding string literal parsing. For example, if the config is
enabled, the `regexp` that can match "\abc" is "^\abc$".
Examples:
When spark.sql.parser.escapedStringLiterals is disabled (default).
> SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*'
true
When spark.sql.parser.escapedStringLiterals is enabled.
> SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\Users.*'
true
See also:
Use LIKE to match with simple string pattern.
""")
case class RLike(left: Expression, right: Expression) extends StringRegexExpression {

override def escape(v: String): String = v
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 @@ -1423,7 +1426,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 @@ -121,8 +122,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)
}

/** 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) remain escaped in our SQL " +
"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 @@ -917,6 +925,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 @@ -418,38 +433,79 @@ class ExpressionParserSuite extends PlanTest {
}

test("strings") {
// 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 :)")
Seq(true, false).foreach { escape =>
val conf = new SQLConf()
conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, escape.toString)
val parser = new CatalystSqlParser(conf)

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

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

// '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%", parser)
assertEqual("'no-pattern\\%'", "no-pattern\\%", parser)

// tests that have different result regarding the conf
if (escape) {
// When SQLConf.ESCAPED_STRING_LITERALS is enabled, string literal parsing fallbacks to
// Spark 1.6 behavior.

// 'LIKE' string literals.
assertEqual("'pattern\\\\%'", "pattern\\\\%", parser)
assertEqual("'pattern\\\\\\%'", "pattern\\\\\\%", parser)

// Escaped characters.
assertEqual("'\0'", "\u0000", parser) // ASCII NUL (X'00')

// 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 '''"))

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

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

This comment has been minimized.

Copy link
@srowen

srowen May 21, 2017

Member

@viirya did you mean to write \ instead of \\ here, unlike in the else branch below? \110 is a deprecated way to just generate the letter "H" and is parsed by scalac so this test doesn't seem to do anything as it is. It's meant to test that the parser can deal with the escapes? I can adjust it to \\110 etc if so in my patch to update to Scala 2.11+ support only because that's what generates the warning.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

\\ in the else branch below is for the 2.0+ parser behavior. Here it is intentional to write \ to follow the 1.6 behavior. Adjusting it to \\110 here should fail this test because it becomes a \ character followed by 110, instead of the letter H.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

If the warning is really needed to be removed, or we may consider remove this line.

This comment has been minimized.

Copy link
@srowen

srowen May 21, 2017

Member

This line is equivalent to writing assertEqual("'Hello!'", "Hello!", parser) in this source code. That doesn't seem to be testing anything about the parser. It never sees anything like \110. Can this be right -- I'm not following.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

It is kind of the same case in above tests, e.g. assertEqual("'\t'", "\t", parser). They are here to compare with the below tests of the parser behavior in 2.0+.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

Maybe a better replacement of this line is something like:

assertEqual("'\\110\\145\\154\\154\\157\\041'", "\\110\\145\\154\\154\\157\\041", parser)

That shows the unescape way of string parsing in 2.0+ can't work if the added flag is enabled. What do you think?

This comment has been minimized.

Copy link
@srowen

srowen May 21, 2017

Member

What is assertEqual("'\t'", "\t", parser) supposed to test? it does not test any parsing of any escape sequence. The string contains no backslash at all. To be clear, is that on purpose? Did you mean to use triple-double quotes? """\t""" is a string of length 2; "\t" is a string of length 1 containing only a tab character. Sorry if you understand this and I'm not understanding your suggestion, but I am not clear we are clear on this point yet.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

Yeah, I think you are right. We shouldn't test this like currently doing.

A better way to test the case the flag is enabled, should be using the 2.0+ unescape string and escaped characters, e.g.:

assertEqual("'\\ttest\t'", "\\ttest\t", parser)

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

If the above test looks ok for you, I can submit a follow-up pr to fix those tests.

This comment has been minimized.

Copy link
@viirya

viirya May 21, 2017

Author Member

We are going to show the unescape string parsing in 2.0+ shouldn't work when the flag is enabled. And we should use escaped characters if we need to specify those special characters.

// Unicode
assertEqual("'\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029'", "World :)", parser)
} else {
// Default behavior

// 'LIKE' string literals.
assertEqual("'pattern\\\\%'", "pattern\\%", parser)
assertEqual("'pattern\\\\\\%'", "pattern\\\\%", parser)

// Escaped characters.
// 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)

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

// Unicode
assertEqual("'\\u0057\\u006F\\u0072\\u006C\\u0064\\u0020\\u003A\\u0029'", "World :)",
parser)
}

}
}

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("SPARK-20399: do not unescaped regex pattern when ESCAPED_STRING_LITERALS is enabled") {
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

0 comments on commit 609ba5f

Please sign in to comment.