aboutsummaryrefslogtreecommitdiff
path: root/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
diff options
context:
space:
mode:
Diffstat (limited to 'sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala50
1 files changed, 31 insertions, 19 deletions
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 cde8bd5b96..ca22ea2420 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)