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

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

jaceklaskowski commented on code in PR #38171:
URL: https://github.com/apache/spark/pull/38171#discussion_r1167999212


##########
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:
   nit: `equalsIgnoreCase`?



##########
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:
   Can `p` be `null`? Why is `Option` required here?



##########
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:
   sealed?



##########
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:
   Why is `Expression` better than `Predicate`?



##########
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's no longer valid given 3.4.0 is out already, isn't 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:
   nit: regex.engine (all lowercase)



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