You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "lyy-pineapple (via GitHub)" <gi...@apache.org> on 2023/04/17 02:59:19 UTC

[GitHub] [spark] lyy-pineapple commented on a diff in pull request #38171: [SPARK-9213] [SQL] Improve regular expression performance (via joni)

lyy-pineapple commented on code in PR #38171:
URL: https://github.com/apache/spark/pull/38171#discussion_r1168109863


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4175,6 +4175,12 @@ object SQLConf {
     .checkValues(ErrorMessageFormat.values.map(_.toString))
     .createWithDefault(ErrorMessageFormat.PRETTY.toString)
 
+  val REGEX_ENGINE =

Review Comment:
   Done



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala:
##########
@@ -498,15 +511,6 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       StringSplit(Literal("\"quote"), Literal("\"quote"), Literal(-1)) :: Nil)
   }
 
-  test("SPARK-30759: cache initialization for literal patterns") {

Review Comment:
   ![image](https://user-images.githubusercontent.com/46274164/232367159-bed99026-5335-4953-bf06-547a41ac1406.png)
   



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4007,3 +4008,20 @@ object RemoveTempResolvedColumn extends Rule[LogicalPlan] {
     }
   }
 }
+
+object RegexSubstitution extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
+    case Like(left, right, escapeChar) if (conf.regexEngine == "joni") =>

Review Comment:
   thanks,i will fit it.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4175,6 +4175,12 @@ object SQLConf {
     .checkValues(ErrorMessageFormat.values.map(_.toString))
     .createWithDefault(ErrorMessageFormat.PRETTY.toString)
 
+  val REGEX_ENGINE =
+    buildConf("spark.sql.Regex.Engine")
+      .version("3.4.0")

Review Comment:
   it should be 3.4.1?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala:
##########
@@ -101,19 +103,55 @@ package object dsl {
       case _ => In(expr, list)
     }
 
-    def like(other: Expression, escapeChar: Char = '\\'): Predicate =
-      Like(expr, other, escapeChar)
+    def like(other: Expression, escapeChar: Char = '\\'): Expression = {

Review Comment:
   Sorry, my mistake. I have restored it



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4175,6 +4175,12 @@ object SQLConf {
     .checkValues(ErrorMessageFormat.values.map(_.toString))
     .createWithDefault(ErrorMessageFormat.PRETTY.toString)
 
+  val REGEX_ENGINE =
+    buildConf("spark.sql.Regex.Engine")

Review Comment:
   Done



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala:
##########
@@ -498,15 +511,6 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       StringSplit(Literal("\"quote"), Literal("\"quote"), Literal(-1)) :: Nil)
   }
 
-  test("SPARK-30759: cache initialization for literal patterns") {

Review Comment:
   It has not been removed and is listed separately. It obtained the pattern in Java regular, but the joni regular engine pattern is private and cannot be obtained.



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala:
##########
@@ -23,9 +23,17 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.{BooleanType, StringType}
 
-class LikeSimplificationSuite extends PlanTest {
+abstract class LikeSimplificationSuite extends PlanTest {

Review Comment:
   Done



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -771,6 +771,35 @@ object LikeSimplification extends Rule[LogicalPlan] with PredicateHelper {
     }
   }
 
+  private def simplifyMultiLikeJoni(
+      child: Expression, patterns: Seq[UTF8String], multi: MultiLikeJoniBase): Expression = {
+    val (remainPatternMap, replacementMap) =
+      patterns.map { p =>
+        p -> Option(p).flatMap(p => simplifyLike(child, p.toString))

Review Comment:
   because of this patch:https://github.com/apache/spark/pull/31976



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4007,3 +4008,20 @@ object RemoveTempResolvedColumn extends Rule[LogicalPlan] {
     }
   }
 }
+
+object RegexSubstitution extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
+    case Like(left, right, escapeChar) if (conf.regexEngine == "joni") =>

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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