From d4028de97687385fa1d1eb6301eb544c0ea4a135 Mon Sep 17 00:00:00 2001
From: Ryan Blue <blue@apache.org>
Date: Wed, 9 Nov 2016 11:00:53 -0800
Subject: [PATCH] [SPARK-18368][SQL] Fix regexp replace when serialized

## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15834 from rdblue/SPARK-18368-fix-regexp-replace.
---
 .../catalyst/expressions/regexpExpressions.scala |  2 +-
 .../expressions/RegexpExpressionsSuite.scala     | 16 +++++++++++++++-
 2 files changed, 16 insertions(+), 2 deletions(-)

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 5648ad6b6d..4896a6225a 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
@@ -230,7 +230,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio
   @transient private var lastReplacement: String = _
   @transient private var lastReplacementInUTF8: UTF8String = _
   // result buffer write by Matcher
-  @transient private val result: StringBuffer = new StringBuffer
+  @transient private lazy val result: StringBuffer = new StringBuffer
 
   override def nullSafeEval(s: Any, p: Any, r: Any): Any = {
     if (!p.equals(lastRegex)) {
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 5299549e7b..d0d1aaa9d2 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
@@ -17,7 +17,8 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.serializer.JavaSerializer
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.types.StringType
 
@@ -191,4 +192,17 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(StringSplit(s1, s2), null, row3)
   }
 
+  test("RegExpReplace serialization") {
+    val serializer = new JavaSerializer(new SparkConf()).newInstance
+
+    val row = create_row("abc", "b", "")
+
+    val s = 's.string.at(0)
+    val p = 'p.string.at(1)
+    val r = 'r.string.at(2)
+
+    val expr: RegExpReplace = serializer.deserialize(serializer.serialize(RegExpReplace(s, p, r)))
+    checkEvaluation(expr, "ac", row)
+  }
+
 }
-- 
GitLab