Skip to content

Commit

Permalink
[SPARK-28083][SQL] Support LIKE ... ESCAPE syntax
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

The syntax 'LIKE predicate: ESCAPE clause' is a ANSI SQL.
For example:

```
select 'abcSpark_13sd' LIKE '%Spark\\_%';             //true
select 'abcSpark_13sd' LIKE '%Spark/_%';              //false
select 'abcSpark_13sd' LIKE '%Spark"_%';              //false
select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/';   //true
select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"';   //true
select 'abcSpark%13sd' LIKE '%Spark\\%%';             //true
select 'abcSpark%13sd' LIKE '%Spark/%%';              //false
select 'abcSpark%13sd' LIKE '%Spark"%%';              //false
select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/';   //true
select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"';   //true
select 'abcSpark\\13sd' LIKE '%Spark\\\\_%';          //true
select 'abcSpark/13sd' LIKE '%Spark//_%';             //false
select 'abcSpark"13sd' LIKE '%Spark""_%';             //false
select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/';  //true
select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"';  //true
```
But Spark SQL only supports 'LIKE predicate'.

Note: If the input string or pattern string is null, then the result is null too.

There are some mainstream database support the syntax.

**PostgreSQL:**
https://www.postgresql.org/docs/11/functions-matching.html

**Vertica:**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Predicates/LIKE-predicate.htm?zoom_highlight=like%20escape

**MySQL:**
https://dev.mysql.com/doc/refman/5.6/en/string-comparison-functions.html

**Oracle:**
https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/JDBC-reference-information.html#GUID-5D371A5B-D7F6-42EB-8C0D-D317F3C53708
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Pattern-matching-Conditions.html#GUID-0779657B-06A8-441F-90C5-044B47862A0A

## How was this patch tested?

Exists UT and new UT.

This PR merged to my production environment and runs above sql:
```
spark-sql> select 'abcSpark_13sd' LIKE '%Spark\\_%';
true
Time taken: 0.119 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%';
false
Time taken: 0.103 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%';
false
Time taken: 0.096 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/';
true
Time taken: 0.096 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"';
true
Time taken: 0.092 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark\\%%';
true
Time taken: 0.109 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%';
false
Time taken: 0.1 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%';
false
Time taken: 0.081 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/';
true
Time taken: 0.095 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"';
true
Time taken: 0.113 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark\\13sd' LIKE '%Spark\\\\_%';
true
Time taken: 0.078 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%';
false
Time taken: 0.067 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%';
false
Time taken: 0.084 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/';
true
Time taken: 0.091 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"';
true
Time taken: 0.091 seconds, Fetched 1 row(s)
```
I create a table and its schema is:
```
spark-sql> desc formatted gja_test;
key     string  NULL
value   string  NULL
other   string  NULL

# Detailed Table Information
Database        test
Table   gja_test
Owner   test
Created Time    Wed Apr 10 11:06:15 CST 2019
Last Access     Thu Jan 01 08:00:00 CST 1970
Created By      Spark 2.4.1-SNAPSHOT
Type    MANAGED
Provider        hive
Table Properties        [transient_lastDdlTime=1563443838]
Statistics      26 bytes
Location        hdfs://namenode.xxx:9000/home/test/hive/warehouse/test.db/gja_test
Serde Library   org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat     org.apache.hadoop.mapred.TextInputFormat
OutputFormat    org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Storage Properties      [field.delim=   , serialization.format= ]
Partition Provider      Catalog
Time taken: 0.642 seconds, Fetched 21 row(s)
```
Table `gja_test` exists three rows of data.
```
spark-sql> select * from gja_test;
a       A       ao
b       B       bo
"__     """__   "
Time taken: 0.665 seconds, Fetched 3 row(s)
```
At finally, I test this function:
```
spark-sql> select * from gja_test where key like value escape '"';
"__     """__   "
Time taken: 0.687 seconds, Fetched 1 row(s)
```

Closes #25001 from beliefer/ansi-sql-like.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
  • Loading branch information
2 people authored and gengliangwang committed Dec 6, 2019
1 parent b86d4bb commit 187f3c1
Show file tree
Hide file tree
Showing 12 changed files with 167 additions and 28 deletions.
1 change: 1 addition & 0 deletions docs/sql-keywords.md
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ Below is a list of all the keywords in Spark SQL.
<tr><td>DROP</td><td>non-reserved</td><td>non-reserved</td><td>reserved</td></tr>
<tr><td>ELSE</td><td>reserved</td><td>non-reserved</td><td>reserved</td></tr>
<tr><td>END</td><td>reserved</td><td>non-reserved</td><td>reserved</td></tr>
<tr><td>ESCAPE</td><td>reserved</td><td>non-reserved</td><td>reserved</td></tr>
<tr><td>ESCAPED</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
<tr><td>EXCEPT</td><td>reserved</td><td>strict-non-reserved</td><td>reserved</td></tr>
<tr><td>EXCHANGE</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -724,7 +724,8 @@ predicate
: NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression
| NOT? kind=IN '(' expression (',' expression)* ')'
| NOT? kind=IN '(' query ')'
| NOT? kind=(RLIKE | LIKE) pattern=valueExpression
| NOT? kind=RLIKE pattern=valueExpression
| NOT? kind=LIKE pattern=valueExpression (ESCAPE escapeChar=STRING)?
| IS NOT? kind=NULL
| IS NOT? kind=(TRUE | FALSE | UNKNOWN)
| IS NOT? kind=DISTINCT FROM right=valueExpression
Expand Down Expand Up @@ -1265,6 +1266,7 @@ nonReserved
| DROP
| ELSE
| END
| ESCAPE
| ESCAPED
| EXCHANGE
| EXISTS
Expand Down Expand Up @@ -1525,6 +1527,7 @@ DISTRIBUTE: 'DISTRIBUTE';
DROP: 'DROP';
ELSE: 'ELSE';
END: 'END';
ESCAPE: 'ESCAPE';
ESCAPED: 'ESCAPED';
EXCEPT: 'EXCEPT';
EXCHANGE: 'EXCHANGE';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,8 @@ package object dsl {
case _ => In(expr, list)
}

def like(other: Expression): Expression = Like(expr, other)
def like(other: Expression, escapeChar: Char = '\\'): Expression =
Like(expr, other, escapeChar)
def rlike(other: Expression): Expression = RLike(expr, other)
def contains(other: Expression): Expression = Contains(expr, other)
def startsWith(other: Expression): Expression = StartsWith(expr, other)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,8 @@ abstract class StringRegexExpression extends BinaryExpression
* Simple RegEx pattern matching function
*/
@ExpressionDescription(
usage = "str _FUNC_ pattern - Returns true if str matches pattern, " +
"null if any arguments are null, false otherwise.",
usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches `pattern` with " +
"`escape`, null if any arguments are null, false otherwise.",
arguments = """
Arguments:
* str - a string expression
Expand All @@ -83,16 +83,15 @@ abstract class StringRegexExpression extends BinaryExpression
% matches zero or 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.
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".
* escape - an character added since Spark 3.0. The default escape character is the '\'.
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 = """
Examples:
Expand All @@ -104,19 +103,25 @@ abstract class StringRegexExpression extends BinaryExpression
spark.sql.parser.escapedStringLiterals false
> SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%\\\\Users%';
true
> SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' ESCAPE '/';
true
""",
note = """
Use RLIKE to match with standard regular expressions.
""",
since = "1.0.0")
// scalastyle:on line.contains.tab
case class Like(left: Expression, right: Expression) extends StringRegexExpression {
case class Like(left: Expression, right: Expression, escapeChar: Char = '\\')
extends StringRegexExpression {

override def escape(v: String): String = StringUtils.escapeLikeRegex(v)
override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar)

override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches()

override def toString: String = s"$left LIKE $right"
override def toString: String = escapeChar match {
case '\\' => s"$left LIKE $right"
case c => s"$left LIKE $right ESCAPE '$c'"
}

override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val patternClass = classOf[Pattern].getName
Expand Down Expand Up @@ -149,10 +154,18 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi
} else {
val pattern = ctx.freshName("pattern")
val rightStr = ctx.freshName("rightStr")
// We need double escape to avoid org.codehaus.commons.compiler.CompileException.
// '\\' will cause exception 'Single quote must be backslash-escaped in character literal'.
// '\"' will cause exception 'Line break in literal not allowed'.
val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') {
s"""\\\\\\$escapeChar"""
} else {
escapeChar
}
nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
s"""
String $rightStr = $eval2.toString();
$patternClass $pattern = $patternClass.compile($escapeFunc($rightStr));
$patternClass $pattern = $patternClass.compile($escapeFunc($rightStr, '$newEscapeChar'));
${ev.value} = $pattern.matcher($eval1.toString()).matches();
"""
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,7 @@ object LikeSimplification extends Rule[LogicalPlan] {
private val equalTo = "([^_%]*)".r

def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case Like(input, Literal(pattern, StringType)) =>
case Like(input, Literal(pattern, StringType), escapeChar) =>
if (pattern == null) {
// If pattern is null, return null value directly, since "col like null" == null.
Literal(null, BooleanType)
Expand All @@ -503,8 +503,7 @@ object LikeSimplification extends Rule[LogicalPlan] {
Contains(input, Literal(infix))
case equalTo(str) =>
EqualTo(input, Literal(str))
case _ =>
Like(input, Literal.create(pattern, StringType))
case _ => Like(input, Literal.create(pattern, StringType), escapeChar)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1386,7 +1386,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
case SqlBaseParser.IN =>
invertIfNotDefined(In(e, ctx.expression.asScala.map(expression)))
case SqlBaseParser.LIKE =>
invertIfNotDefined(Like(e, expression(ctx.pattern)))
val escapeChar = Option(ctx.escapeChar).map(string).map { str =>
if (str.length != 1) {
throw new ParseException("Invalid escape string." +
"Escape string must contains only one character.", ctx)
}
str.charAt(0)
}.getOrElse('\\')
invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar))
case SqlBaseParser.RLIKE =>
invertIfNotDefined(RLike(e, expression(ctx.pattern)))
case SqlBaseParser.NULL if ctx.NOT != null =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,10 @@ object StringUtils extends Logging {
* throw an [[AnalysisException]].
*
* @param pattern the SQL pattern to convert
* @param escapeStr the escape string contains one character.
* @return the equivalent Java regular expression of the pattern
*/
def escapeLikeRegex(pattern: String): String = {
def escapeLikeRegex(pattern: String, escapeChar: Char): String = {
val in = pattern.toIterator
val out = new StringBuilder()

Expand All @@ -50,13 +51,14 @@ object StringUtils extends Logging {

while (in.hasNext) {
in.next match {
case '\\' if in.hasNext =>
case c1 if c1 == escapeChar && in.hasNext =>
val c = in.next
c match {
case '_' | '%' | '\\' => out ++= Pattern.quote(Character.toString(c))
case '_' | '%' => out ++= Pattern.quote(Character.toString(c))
case c if c == escapeChar => out ++= Pattern.quote(Character.toString(c))
case _ => fail(s"the escape character is not allowed to precede '$c'")
}
case '\\' => fail("it is not allowed to end with the escape character")
case c if c == escapeChar => fail("it is not allowed to end with the escape character")
case '_' => out ++= "."
case '%' => out ++= ".*"
case c => out ++= Pattern.quote(Character.toString(c))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,84 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkLiteralRow("""%SystemDrive%\Users\John""" like _, """\%SystemDrive\%\\Users%""", true)
}

Seq('/', '#', '\"').foreach { escapeChar =>
test(s"LIKE Pattern ESCAPE '$escapeChar'") {
// null handling
checkLiteralRow(Literal.create(null, StringType).like(_, escapeChar), "a", null)
checkEvaluation(
Literal.create("a", StringType).like(Literal.create(null, StringType), escapeChar), null)
checkEvaluation(
Literal.create(null, StringType).like(Literal.create(null, StringType), escapeChar), null)
checkEvaluation(Literal.create("a", StringType).like(
NonFoldableLiteral.create("a", StringType), escapeChar), true)
checkEvaluation(Literal.create("a", StringType).like(
NonFoldableLiteral.create(null, StringType), escapeChar), null)
checkEvaluation(Literal.create(null, StringType).like(
NonFoldableLiteral.create("a", StringType), escapeChar), null)
checkEvaluation(Literal.create(null, StringType).like(
NonFoldableLiteral.create(null, StringType), escapeChar), null)

// simple patterns
checkLiteralRow("abdef" like(_, escapeChar), "abdef", true)
checkLiteralRow("a_%b" like(_, escapeChar), s"a${escapeChar}__b", true)
checkLiteralRow("addb" like(_, escapeChar), "a_%b", true)
checkLiteralRow("addb" like(_, escapeChar), s"a${escapeChar}__b", false)
checkLiteralRow("addb" like(_, escapeChar), s"a%$escapeChar%b", false)
checkLiteralRow("a_%b" like(_, escapeChar), s"a%$escapeChar%b", true)
checkLiteralRow("addb" like(_, escapeChar), "a%", true)
checkLiteralRow("addb" like(_, escapeChar), "**", false)
checkLiteralRow("abc" like(_, escapeChar), "a%", true)
checkLiteralRow("abc" like(_, escapeChar), "b%", false)
checkLiteralRow("abc" like(_, escapeChar), "bc%", false)
checkLiteralRow("a\nb" like(_, escapeChar), "a_b", true)
checkLiteralRow("ab" like(_, escapeChar), "a%b", true)
checkLiteralRow("a\nb" like(_, escapeChar), "a%b", true)

// empty input
checkLiteralRow("" like(_, escapeChar), "", true)
checkLiteralRow("a" like(_, escapeChar), "", false)
checkLiteralRow("" like(_, escapeChar), "a", false)

// SI-17647 double-escaping backslash
checkLiteralRow(s"""$escapeChar$escapeChar$escapeChar$escapeChar""" like(_, escapeChar),
s"""%$escapeChar$escapeChar%""", true)
checkLiteralRow("""%%""" like(_, escapeChar), """%%""", true)
checkLiteralRow(s"""${escapeChar}__""" like(_, escapeChar),
s"""$escapeChar$escapeChar${escapeChar}__""", true)
checkLiteralRow(s"""$escapeChar$escapeChar${escapeChar}__""" like(_, escapeChar),
s"""%$escapeChar$escapeChar%$escapeChar%""", false)
checkLiteralRow(s"""_$escapeChar$escapeChar$escapeChar%""" like(_, escapeChar),
s"""%$escapeChar${escapeChar}""", false)

// unicode
// scalastyle:off nonascii
checkLiteralRow("a\u20ACa" like(_, escapeChar), "_\u20AC_", true)
checkLiteralRow("a€a" like(_, escapeChar), "_€_", true)
checkLiteralRow("a€a" like(_, escapeChar), "_\u20AC_", true)
checkLiteralRow("a\u20ACa" like(_, escapeChar), "_€_", true)
// scalastyle:on nonascii

// invalid escaping
val invalidEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like(s"""${escapeChar}a""", escapeChar))
}
assert(invalidEscape.getMessage.contains("pattern"))
val endEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like(s"""a$escapeChar""", escapeChar))
}
assert(endEscape.getMessage.contains("pattern"))

// case
checkLiteralRow("A" like(_, escapeChar), "a%", false)
checkLiteralRow("a" like(_, escapeChar), "A%", false)
checkLiteralRow("AaA" like(_, escapeChar), "_a_", true)

// example
checkLiteralRow(s"""%SystemDrive%${escapeChar}Users${escapeChar}John""" like(_, escapeChar),
s"""$escapeChar%SystemDrive$escapeChar%$escapeChar${escapeChar}Users%""", true)
}
}

test("RLIKE Regular Expression") {
checkLiteralRow(Literal.create(null, StringType) rlike _, "abdef", null)
checkEvaluation("abdef" rlike Literal.create(null, StringType), null)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,18 @@ class ExpressionParserSuite extends AnalysisTest {
assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
}

test("like escape expressions") {
val message = "Escape string must contains only one character."
assertEqual("a like 'pattern%' escape '#'", 'a.like("pattern%", '#'))
assertEqual("a like 'pattern%' escape '\"'", 'a.like("pattern%", '\"'))
intercept("a like 'pattern%' escape '##'", message)
intercept("a like 'pattern%' escape ''", message)
assertEqual("a not like 'pattern%' escape '#'", !('a.like("pattern%", '#')))
assertEqual("a not like 'pattern%' escape '\"'", !('a.like("pattern%", '\"')))
intercept("a not like 'pattern%' escape '\"/'", message)
intercept("a not like 'pattern%' escape ''", message)
}

test("like expressions with ESCAPED_STRING_LITERALS = true") {
val conf = new SQLConf()
conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -367,6 +367,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper {
"drop",
"else",
"end",
"escape",
"escaped",
"except",
"exchange",
Expand Down Expand Up @@ -581,6 +582,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper {
"distinct",
"else",
"end",
"escape",
"except",
"false",
"fetch",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,34 @@ import org.apache.spark.sql.catalyst.util.StringUtils._
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\\Q_\\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")
val expectedEscapedStrOne = "(?s)\\Qa\\E\\Qb\\E\\Qd\\E\\Qe\\E\\Qf\\E"
val expectedEscapedStrTwo = "(?s)\\Qa\\E\\Q_\\E.\\Qb\\E"
val expectedEscapedStrThree = "(?s)\\Qa\\E..*\\Qb\\E"
val expectedEscapedStrFour = "(?s)\\Qa\\E.*\\Q%\\E\\Qb\\E"
val expectedEscapedStrFive = "(?s)\\Qa\\E.*"
val expectedEscapedStrSix = "(?s)\\Q*\\E\\Q*\\E"
val expectedEscapedStrSeven = "(?s)\\Qa\\E.\\Qb\\E"
assert(escapeLikeRegex("abdef", '\\') === expectedEscapedStrOne)
assert(escapeLikeRegex("abdef", '/') === expectedEscapedStrOne)
assert(escapeLikeRegex("abdef", '\"') === expectedEscapedStrOne)
assert(escapeLikeRegex("a\\__b", '\\') === expectedEscapedStrTwo)
assert(escapeLikeRegex("a/__b", '/') === expectedEscapedStrTwo)
assert(escapeLikeRegex("a\"__b", '\"') === expectedEscapedStrTwo)
assert(escapeLikeRegex("a_%b", '\\') === expectedEscapedStrThree)
assert(escapeLikeRegex("a_%b", '/') === expectedEscapedStrThree)
assert(escapeLikeRegex("a_%b", '\"') === expectedEscapedStrThree)
assert(escapeLikeRegex("a%\\%b", '\\') === expectedEscapedStrFour)
assert(escapeLikeRegex("a%/%b", '/') === expectedEscapedStrFour)
assert(escapeLikeRegex("a%\"%b", '\"') === expectedEscapedStrFour)
assert(escapeLikeRegex("a%", '\\') === expectedEscapedStrFive)
assert(escapeLikeRegex("a%", '/') === expectedEscapedStrFive)
assert(escapeLikeRegex("a%", '\"') === expectedEscapedStrFive)
assert(escapeLikeRegex("**", '\\') === expectedEscapedStrSix)
assert(escapeLikeRegex("**", '/') === expectedEscapedStrSix)
assert(escapeLikeRegex("**", '\"') === expectedEscapedStrSix)
assert(escapeLikeRegex("a_b", '\\') === expectedEscapedStrSeven)
assert(escapeLikeRegex("a_b", '/') === expectedEscapedStrSeven)
assert(escapeLikeRegex("a_b", '\"') === expectedEscapedStrSeven)
}

test("filter pattern") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper {
case Not(expr) => isLikelySelective(expr)
case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
case Like(_, _) => true
case Like(_, _, _) => true
case _: BinaryComparison => true
case _: In | _: InSet => true
case _: StringPredicate => true
Expand Down

0 comments on commit 187f3c1

Please sign in to comment.