You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/07/30 13:25:27 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #27507: [SPARK-24884][SQL] Support regexp function regexp_extract_all

cloud-fan commented on a change in pull request #27507:
URL: https://github.com/apache/spark/pull/27507#discussion_r462993465



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##########
@@ -421,20 +423,59 @@ object RegExpExtract {
   }
 }
 
+abstract class RegExpExtractBase
+  extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant {
+  def subject: Expression
+  def regexp: Expression
+  def idx: Expression
+
+  // last regex in string, we will update the pattern iff regexp value changed.
+  @transient private var lastRegex: UTF8String = _
+  // last regex pattern, we cache it for performance concern
+  @transient private var pattern: Pattern = _
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType)
+  override def children: Seq[Expression] = subject :: regexp :: idx :: Nil
+
+  protected def getLastMatcher(s: Any, p: Any): Matcher = {
+    if (!p.equals(lastRegex)) {
+      // regex value changed
+      lastRegex = p.asInstanceOf[UTF8String].clone()
+      pattern = Pattern.compile(lastRegex.toString)
+    }
+    pattern.matcher(s.toString)
+  }
+}
+
 /**
  * Extract a specific(idx) group identified by a Java regex.
  *
  * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status.
  */
 @ExpressionDescription(
-  usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.",
+  usage = """
+    _FUNC_(str, regexp[, idx]) - Extract the first string in the `str` that match the `regexp`
+    expression and corresponding to the regex group index.
+  """,
   arguments = """
     Arguments:
       * str - a string expression.
-      * regexp - a string representing a regular expression.
-          The regex string should be a Java regular expression.
-      * idx - an integer expression that representing the group index. The group index should be
-          non-negative. If `idx` is not specified, the default group index value is 1.
+      * regexp - a string representing a regular expression. The regex 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$".
+      * idx - an integer expression that representing the group index. The regex maybe contains
+          multiple groups. `idx` indicates which regex group to extract. The group index should
+          be non-negative. If `idx` is not specified, the default group index value is 1. The

Review comment:
       It's important to mention the start point. Seems the `idx` starts with 1 not 0?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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