You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2017/04/17 18:18:00 UTC

spark git commit: [SPARK-17647][SQL] Fix backslash escaping in 'LIKE' patterns.

Repository: spark
Updated Branches:
  refs/heads/master 01ff0350a -> e5fee3e4f


[SPARK-17647][SQL] Fix backslash escaping in 'LIKE' patterns.

## What changes were proposed in this pull request?

This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping.
A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead.

---

Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features.

| RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity |
| --- | --- | --- | --- |
| [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no |
| [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes |
| [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes |
| [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no |
| [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes |
| [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes |
| Current Spark | _, % | \ | yes |

[1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket.

The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed.
1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`.
   PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid.
   _Proposed new behaviour in Spark: throw AnalysisException_
2. [x] Empty input, e.g. `'' like ''`
   Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule.
3. [x] Escape before a non-special character, e.g. `'a' like '\a'`.
   Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign.
   _Proposed new behaviour in Spark: throw AnalysisException_

The current specification is also described in the operator's source code in this patch.
## How was this patch tested?

Extra case in regex unit tests.

Author: Jakob Odersky <ja...@odersky.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rx...@databricks.com>

Closes #15398 from jodersky/SPARK-17647.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e5fee3e4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e5fee3e4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e5fee3e4

Branch: refs/heads/master
Commit: e5fee3e4f853f906f0b476bb04ee35a15f1ae650
Parents: 01ff035
Author: Jakob Odersky <ja...@odersky.com>
Authored: Mon Apr 17 11:17:57 2017 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Mon Apr 17 11:17:57 2017 -0700

----------------------------------------------------------------------
 .../expressions/regexpExpressions.scala         |  25 ++-
 .../spark/sql/catalyst/util/StringUtils.scala   |  50 +++---
 .../expressions/RegexpExpressionsSuite.scala    | 161 +++++++++++--------
 .../sql/catalyst/util/StringUtilsSuite.scala    |   4 +-
 4 files changed, 153 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/e5fee3e4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 49b7797..a36da8e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -69,7 +69,30 @@ abstract class StringRegexExpression extends BinaryExpression
  * Simple RegEx pattern matching function
  */
 @ExpressionDescription(
-  usage = "str _FUNC_ pattern - Returns true if `str` matches `pattern`, or false otherwise.")
+  usage = "str _FUNC_ pattern - Returns true if str matches pattern, " +
+    "null if any arguments are null, false otherwise.",
+  extended = """
+    Arguments:
+      str - a string expression
+      pattern - a string expression. The pattern is a string which is matched literally, with
+        exception to the following special symbols:
+
+          _ matches any one character in the input (similar to . in posix regular expressions)
+
+          % matches zero ore more characters in the input (similar to .* in posix regular
+          expressions)
+
+        The escape character is '\'. If an escape character precedes a special symbol or another
+        escape character, the following character is matched literally. It is invalid to escape
+        any other character.
+
+    Examples:
+      > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%'
+      true
+
+    See also:
+      Use RLIKE to match with standard regular expressions.
+""")
 case class Like(left: Expression, right: Expression) extends StringRegexExpression {
 
   override def escape(v: String): String = StringUtils.escapeLikeRegex(v)

http://git-wip-us.apache.org/repos/asf/spark/blob/e5fee3e4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
index cde8bd5..ca22ea2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
@@ -19,32 +19,44 @@ package org.apache.spark.sql.catalyst.util
 
 import java.util.regex.{Pattern, PatternSyntaxException}
 
+import org.apache.spark.sql.AnalysisException
 import org.apache.spark.unsafe.types.UTF8String
 
 object StringUtils {
 
-  // replace the _ with .{1} exactly match 1 time of any character
-  // replace the % with .*, match 0 or more times with any character
-  def escapeLikeRegex(v: String): String = {
-    if (!v.isEmpty) {
-      "(?s)" + (' ' +: v.init).zip(v).flatMap {
-        case (prev, '\\') => ""
-        case ('\\', c) =>
-          c match {
-            case '_' => "_"
-            case '%' => "%"
-            case _ => Pattern.quote("\\" + c)
-          }
-        case (prev, c) =>
+  /**
+   * Validate and convert SQL 'like' pattern to a Java regular expression.
+   *
+   * Underscores (_) are converted to '.' and percent signs (%) are converted to '.*', other
+   * characters are quoted literally. Escaping is done according to the rules specified in
+   * [[org.apache.spark.sql.catalyst.expressions.Like]] usage documentation. An invalid pattern will
+   * throw an [[AnalysisException]].
+   *
+   * @param pattern the SQL pattern to convert
+   * @return the equivalent Java regular expression of the pattern
+   */
+  def escapeLikeRegex(pattern: String): String = {
+    val in = pattern.toIterator
+    val out = new StringBuilder()
+
+    def fail(message: String) = throw new AnalysisException(
+      s"the pattern '$pattern' is invalid, $message")
+
+    while (in.hasNext) {
+      in.next match {
+        case '\\' if in.hasNext =>
+          val c = in.next
           c match {
-            case '_' => "."
-            case '%' => ".*"
-            case _ => Pattern.quote(Character.toString(c))
+            case '_' | '%' | '\\' => out ++= Pattern.quote(Character.toString(c))
+            case _ => fail(s"the escape character is not allowed to precede '$c'")
           }
-      }.mkString
-    } else {
-      v
+        case '\\' => fail("it is not allowed to end with the escape character")
+        case '_' => out ++= "."
+        case '%' => out ++= ".*"
+        case c => out ++= Pattern.quote(Character.toString(c))
+      }
     }
+    "(?s)" + out.result() // (?s) enables dotall mode, causing "." to match new lines
   }
 
   private[this] val trueStrings = Set("t", "true", "y", "yes", "1").map(UTF8String.fromString)

http://git-wip-us.apache.org/repos/asf/spark/blob/e5fee3e4/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
index 5299549..1ce150e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
@@ -18,16 +18,38 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.types.StringType
+import org.apache.spark.sql.types.{IntegerType, StringType}
 
 /**
  * Unit tests for regular expression (regexp) related SQL expressions.
  */
 class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
 
-  test("LIKE literal Regular Expression") {
-    checkEvaluation(Literal.create(null, StringType).like("a"), null)
+  /**
+   * Check if a given expression evaluates to an expected output, in case the input is
+   * a literal and in case the input is in the form of a row.
+   * @tparam A type of input
+   * @param mkExpr the expression to test for a given input
+   * @param input value that will be used to create the expression, as literal and in the form
+   *        of a row
+   * @param expected the expected output of the expression
+   * @param inputToExpression an implicit conversion from the input type to its corresponding
+   *        sql expression
+   */
+  def checkLiteralRow[A](mkExpr: Expression => Expression, input: A, expected: Any)
+    (implicit inputToExpression: A => Expression): Unit = {
+    checkEvaluation(mkExpr(input), expected) // check literal input
+
+    val regex = 'a.string.at(0)
+    checkEvaluation(mkExpr(regex), expected, create_row(input)) // check row input
+  }
+
+  test("LIKE Pattern") {
+
+    // null handling
+    checkLiteralRow(Literal.create(null, StringType).like(_), "a", null)
     checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null)
     checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null)
     checkEvaluation(
@@ -39,45 +61,64 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(
       Literal.create(null, StringType).like(NonFoldableLiteral.create(null, StringType)), null)
 
-    checkEvaluation("abdef" like "abdef", true)
-    checkEvaluation("a_%b" like "a\\__b", true)
-    checkEvaluation("addb" like "a_%b", true)
-    checkEvaluation("addb" like "a\\__b", false)
-    checkEvaluation("addb" like "a%\\%b", false)
-    checkEvaluation("a_%b" like "a%\\%b", true)
-    checkEvaluation("addb" like "a%", true)
-    checkEvaluation("addb" like "**", false)
-    checkEvaluation("abc" like "a%", true)
-    checkEvaluation("abc"  like "b%", false)
-    checkEvaluation("abc"  like "bc%", false)
-    checkEvaluation("a\nb" like "a_b", true)
-    checkEvaluation("ab" like "a%b", true)
-    checkEvaluation("a\nb" like "a%b", true)
-  }
+    // simple patterns
+    checkLiteralRow("abdef" like _, "abdef", true)
+    checkLiteralRow("a_%b" like _, "a\\__b", true)
+    checkLiteralRow("addb" like _, "a_%b", true)
+    checkLiteralRow("addb" like _, "a\\__b", false)
+    checkLiteralRow("addb" like _, "a%\\%b", false)
+    checkLiteralRow("a_%b" like _, "a%\\%b", true)
+    checkLiteralRow("addb" like _, "a%", true)
+    checkLiteralRow("addb" like _, "**", false)
+    checkLiteralRow("abc" like _, "a%", true)
+    checkLiteralRow("abc"  like _, "b%", false)
+    checkLiteralRow("abc"  like _, "bc%", false)
+    checkLiteralRow("a\nb" like _, "a_b", true)
+    checkLiteralRow("ab" like _, "a%b", true)
+    checkLiteralRow("a\nb" like _, "a%b", true)
+
+    // empty input
+    checkLiteralRow("" like _, "", true)
+    checkLiteralRow("a" like _, "", false)
+    checkLiteralRow("" like _, "a", false)
+
+    // SI-17647 double-escaping backslash
+    checkLiteralRow("""\\\\""" like _, """%\\%""", true)
+    checkLiteralRow("""%%""" like _, """%%""", true)
+    checkLiteralRow("""\__""" like _, """\\\__""", true)
+    checkLiteralRow("""\\\__""" like _, """%\\%\%""", false)
+    checkLiteralRow("""_\\\%""" like _, """%\\""", false)
+
+    // unicode
+    // scalastyle:off nonascii
+    checkLiteralRow("a\u20ACa" like _, "_\u20AC_", true)
+    checkLiteralRow("a\u20aca" like _, "_\u20ac_", true)
+    checkLiteralRow("a\u20aca" like _, "_\u20AC_", true)
+    checkLiteralRow("a\u20ACa" like _, "_\u20ac_", true)
+    // scalastyle:on nonascii
+
+    // invalid escaping
+    val invalidEscape = intercept[AnalysisException] {
+      evaluate("""a""" like """\a""")
+    }
+    assert(invalidEscape.getMessage.contains("pattern"))
+
+    val endEscape = intercept[AnalysisException] {
+      evaluate("""a""" like """a\""")
+    }
+    assert(endEscape.getMessage.contains("pattern"))
+
+    // case
+    checkLiteralRow("A" like _, "a%", false)
+    checkLiteralRow("a" like _, "A%", false)
+    checkLiteralRow("AaA" like _, "_a_", true)
 
-  test("LIKE Non-literal Regular Expression") {
-    val regEx = 'a.string.at(0)
-    checkEvaluation("abcd" like regEx, null, create_row(null))
-    checkEvaluation("abdef" like regEx, true, create_row("abdef"))
-    checkEvaluation("a_%b" like regEx, true, create_row("a\\__b"))
-    checkEvaluation("addb" like regEx, true, create_row("a_%b"))
-    checkEvaluation("addb" like regEx, false, create_row("a\\__b"))
-    checkEvaluation("addb" like regEx, false, create_row("a%\\%b"))
-    checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b"))
-    checkEvaluation("addb" like regEx, true, create_row("a%"))
-    checkEvaluation("addb" like regEx, false, create_row("**"))
-    checkEvaluation("abc" like regEx, true, create_row("a%"))
-    checkEvaluation("abc" like regEx, false, create_row("b%"))
-    checkEvaluation("abc" like regEx, false, create_row("bc%"))
-    checkEvaluation("a\nb" like regEx, true, create_row("a_b"))
-    checkEvaluation("ab" like regEx, true, create_row("a%b"))
-    checkEvaluation("a\nb" like regEx, true, create_row("a%b"))
-
-    checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%"))
+    // example
+    checkLiteralRow("""%SystemDrive%\Users\John""" like _, """\%SystemDrive\%\\Users%""", true)
   }
 
-  test("RLIKE literal Regular Expression") {
-    checkEvaluation(Literal.create(null, StringType) rlike "abdef", null)
+  test("RLIKE Regular Expression") {
+    checkLiteralRow(Literal.create(null, StringType) rlike _, "abdef", null)
     checkEvaluation("abdef" rlike Literal.create(null, StringType), null)
     checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null)
     checkEvaluation("abdef" rlike NonFoldableLiteral.create("abdef", StringType), true)
@@ -87,42 +128,32 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(
       Literal.create(null, StringType) rlike NonFoldableLiteral.create(null, StringType), null)
 
-    checkEvaluation("abdef" rlike "abdef", true)
-    checkEvaluation("abbbbc" rlike "a.*c", true)
+    checkLiteralRow("abdef" rlike _, "abdef", true)
+    checkLiteralRow("abbbbc" rlike _, "a.*c", true)
 
-    checkEvaluation("fofo" rlike "^fo", true)
-    checkEvaluation("fo\no" rlike "^fo\no$", true)
-    checkEvaluation("Bn" rlike "^Ba*n", true)
-    checkEvaluation("afofo" rlike "fo", true)
-    checkEvaluation("afofo" rlike "^fo", false)
-    checkEvaluation("Baan" rlike "^Ba?n", false)
-    checkEvaluation("axe" rlike "pi|apa", false)
-    checkEvaluation("pip" rlike "^(pi)*$", false)
+    checkLiteralRow("fofo" rlike _, "^fo", true)
+    checkLiteralRow("fo\no" rlike _, "^fo\no$", true)
+    checkLiteralRow("Bn" rlike _, "^Ba*n", true)
+    checkLiteralRow("afofo" rlike _, "fo", true)
+    checkLiteralRow("afofo" rlike _, "^fo", false)
+    checkLiteralRow("Baan" rlike _, "^Ba?n", false)
+    checkLiteralRow("axe" rlike _, "pi|apa", false)
+    checkLiteralRow("pip" rlike _, "^(pi)*$", false)
 
-    checkEvaluation("abc"  rlike "^ab", true)
-    checkEvaluation("abc"  rlike "^bc", false)
-    checkEvaluation("abc"  rlike "^ab", true)
-    checkEvaluation("abc"  rlike "^bc", false)
+    checkLiteralRow("abc"  rlike _, "^ab", true)
+    checkLiteralRow("abc"  rlike _, "^bc", false)
+    checkLiteralRow("abc"  rlike _, "^ab", true)
+    checkLiteralRow("abc"  rlike _, "^bc", false)
 
     intercept[java.util.regex.PatternSyntaxException] {
       evaluate("abbbbc" rlike "**")
     }
-  }
-
-  test("RLIKE Non-literal Regular Expression") {
-    val regEx = 'a.string.at(0)
-    checkEvaluation("abdef" rlike regEx, true, create_row("abdef"))
-    checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c"))
-    checkEvaluation("fofo" rlike regEx, true, create_row("^fo"))
-    checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$"))
-    checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n"))
-
     intercept[java.util.regex.PatternSyntaxException] {
-      evaluate("abbbbc" rlike regEx, create_row("**"))
+      val regex = 'a.string.at(0)
+      evaluate("abbbbc" rlike regex, create_row("**"))
     }
   }
 
-
   test("RegexReplace") {
     val row1 = create_row("100-200", "(\\d+)", "num")
     val row2 = create_row("100-200", "(\\d+)", "###")

http://git-wip-us.apache.org/repos/asf/spark/blob/e5fee3e4/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
index 2ffc18a..78fee51 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala
@@ -24,9 +24,9 @@ class StringUtilsSuite extends SparkFunSuite {
 
   test("escapeLikeRegex") {
     assert(escapeLikeRegex("abdef") === "(?s)\\Qa\\E\\Qb\\E\\Qd\\E\\Qe\\E\\Qf\\E")
-    assert(escapeLikeRegex("a\\__b") === "(?s)\\Qa\\E_.\\Qb\\E")
+    assert(escapeLikeRegex("a\\__b") === "(?s)\\Qa\\E\\Q_\\E.\\Qb\\E")
     assert(escapeLikeRegex("a_%b") === "(?s)\\Qa\\E..*\\Qb\\E")
-    assert(escapeLikeRegex("a%\\%b") === "(?s)\\Qa\\E.*%\\Qb\\E")
+    assert(escapeLikeRegex("a%\\%b") === "(?s)\\Qa\\E.*\\Q%\\E\\Qb\\E")
     assert(escapeLikeRegex("a%") === "(?s)\\Qa\\E.*")
     assert(escapeLikeRegex("**") === "(?s)\\Q*\\E\\Q*\\E")
     assert(escapeLikeRegex("a_b") === "(?s)\\Qa\\E.\\Qb\\E")


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org