MaxGekk commented on code in PR #48379:
URL: https://github.com/apache/spark/pull/48379#discussion_r1802690163


##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -2907,6 +2907,12 @@
     ],
     "sqlState" : "42613"
   },
+  "INVALID_REGEXP_REPLACE" : {
+    "message" : [
+      "Could not perform regexp_replace for `source = \"<source>\"`, `pattern 
= \"<pattern>\"`, `replacement = \"<replacement>\"` and `position = 
<position>`."

Review Comment:
   Please, remove backticks. They are not necessary here, and we use them to 
highlight message parameters, literals and so on.
   



##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -2907,6 +2907,12 @@
     ],
     "sqlState" : "42613"
   },
+  "INVALID_REGEXP_REPLACE" : {
+    "message" : [
+      "Could not perform regexp_replace for `source = \"<source>\"`, `pattern 
= \"<pattern>\"`, `replacement = \"<replacement>\"` and `position = 
<position>`."

Review Comment:
   ```suggestion
         "Could not perform `regexp_replace` for source = \"<source>\", pattern 
= \"<pattern>\", replacement = \"<replacement>\" and position = <position>."
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala:
##########
@@ -96,6 +98,31 @@ class CollationSQLRegexpSuite
     }
   }
 
+  test("RegExpReplace throws the right exception when replace fails on a 
particular row") {
+    val tableName = "regexpReplaceException"
+    withTable(tableName) {
+      Seq("NO_CODEGEN", "CODEGEN_ONLY").foreach { codegenMode =>
+        withSQLConf("spark.sql.codegen.factoryMode" -> codegenMode) {
+          sql(s"CREATE TABLE IF NOT EXISTS $tableName(s STRING)")
+          sql(s"INSERT INTO $tableName VALUES('first last')")
+          val query = s"SELECT regexp_replace(s, '(?<first>[a-zA-Z]+) 
(?<last>[a-zA-Z]+)', " +
+            s"'$$3 $$1') FROM $tableName"
+          val df = sql(query)
+          val plan = df.queryExecution.executedPlan
+          assert(plan.isInstanceOf[WholeStageCodegenExec] == (codegenMode == 
"CODEGEN_ONLY"))
+          val exception = intercept[SparkRuntimeException] {
+            df.collect()
+          }
+          assert(exception.getCondition == "INVALID_REGEXP_REPLACE")
+          assert(exception.getMessage.contains("""Could not perform 
regexp_replace for """ +
+            """`source = "first last"`, `pattern = "(?<first>[a-zA-Z]+) 
(?<last>[a-zA-Z]+)"`, """ +
+            """`replacement = "$3 $1"` and `position = 1`."""))

Review Comment:
   Could you use `checkError`, please. We try to avoid embedding error messages 
to source code because of:
   - tech editors and others should be able to edit error formats without 
repackaging Spark
   - we might have error messages in other languages in the future.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala:
##########
@@ -700,7 +700,13 @@ case class RegExpReplace(subject: Expression, regexp: 
Expression, rep: Expressio
       m.region(position, source.length)
       result.delete(0, result.length())
       while (m.find) {
-        m.appendReplacement(result, lastReplacement)
+        try {
+          m.appendReplacement(result, lastReplacement)
+        } catch {
+          case e: Exception =>

Review Comment:
   can't you catch more specific exception here?



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to