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 2022/08/26 08:10:38 UTC

[GitHub] [spark] wangyum opened a new pull request, #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

wangyum opened a new pull request, #37672:
URL: https://github.com/apache/spark/pull/37672

   ### What changes were proposed in this pull request?
   
   Do not simplify multiLike if child is not attribute.
   
   ### Why are the changes needed?
   
   Simplifying multiLike can not benefit the query because it cannot be pushed down. For example:
   ```sql
   select * from t1 where substr(name, 1, 5) like any('%a', 'b%', '%c%');
   ```
   ```
   == Physical Plan ==
   *(1) Filter ((EndsWith(substr(name#0, 1, 5), a) OR StartsWith(substr(name#0, 1, 5), b)) OR Contains(substr(name#0, 1, 5), c))
   +- *(1) ColumnarToRow
      +- FileScan parquet default.t1[name#0] Batched: true, DataFilters: [((EndsWith(substr(name#0, 1, 5), a) OR StartsWith(substr(name#0, 1, 5), b)) OR Contains(substr(n..., Format: Parquet, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<name:string>
   ```
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   Unit test.


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1230023807

   Yes. For example:
   ```scala
   spark.range(1).selectExpr("cast(id as string) as id").write.saveAsTable("t1")
   var i = 0
   spark.udf.register("testHeavyUdf", (s: String) => {
     i = i + 1
     System.out.println("testHeavyUdf: " + i)
     Thread.sleep(1000)
     s
   })
   
   spark.sql("select * from t1 where testHeavyUdf(id) like any('%a', 'b%', '%c%');").collect()
   ```
   
   Before this PR:
   ```
   testHeavyUdf: 1
   testHeavyUdf: 2
   testHeavyUdf: 3
   ```
   After this PR:
   ```
   testHeavyUdf: 1
   ```
   


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1231050298

   It is 3.1.1. https://issues.apache.org/jira/browse/SPARK-33938


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


[GitHub] [spark] cloud-fan commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1231038564

   which Spark version starts to have multi-like optimization?


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1235590339

   Merged to master.


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957921008


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -1147,7 +1147,7 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper {
   /**
    * Check if the given expression is cheap that we can inline it.
    */
-  private def isCheap(e: Expression): Boolean = e match {
+  def isCheap(e: Expression): Boolean = e match {

Review Comment:
   Note that `isCheap` returns true for `PythonUDF`. It's OK for `CollapseProject` as the python udf planner will deduplicate python udfs in a single project operator. I'm not sure about multi-like optimization though.



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


[GitHub] [spark] wangyum closed pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum closed pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression
URL: https://github.com/apache/spark/pull/37672


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957921650


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -1147,7 +1147,7 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper {
   /**
    * Check if the given expression is cheap that we can inline it.
    */
-  private def isCheap(e: Expression): Boolean = e match {
+  def isCheap(e: Expression): Boolean = e match {

Review Comment:
   nvm, we split multi-like into multiple string predicates and they are still in the same operator. python udf should be fine.



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


[GitHub] [spark] beliefer commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
beliefer commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957148956


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -780,10 +780,10 @@ object LikeSimplification extends Rule[LogicalPlan] {
       } else {
         simplifyLike(input, pattern.toString, escapeChar).getOrElse(l)
       }
-    case l @ LikeAll(child, patterns) => simplifyMultiLike(child, patterns, l)
-    case l @ NotLikeAll(child, patterns) => simplifyMultiLike(child, patterns, l)
-    case l @ LikeAny(child, patterns) => simplifyMultiLike(child, patterns, l)
-    case l @ NotLikeAny(child, patterns) => simplifyMultiLike(child, patterns, l)
+    case l @ LikeAll(child: Attribute, patterns) => simplifyMultiLike(child, patterns, l)

Review Comment:
   If child is foldable, it seems we can still simplify it.



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


[GitHub] [spark] cloud-fan commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1238110417

   That commit is too risky. How about we just copy-paste the `isCheap` function when backporting this commit?


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


[GitHub] [spark] cloud-fan commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1229863819

   what was the motivation to optimize multi-like before?


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


[GitHub] [spark] cloud-fan commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1229970689

   that said, it makes the plan worse if we can't push down the predicates eventually?


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1237948216

   Do we need to backport https://issues.apache.org/jira/browse/SPARK-39699 to branch 3.1-3.3? This PR need `isCheap`:
   https://github.com/apache/spark/pull/37165/files#diff-11264d807efa58054cca2d220aae8fba644ee0f0f2a4722c46d52828394846efR1124


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1229686310

   @beliefer @cloud-fan 


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957285643


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -771,6 +771,9 @@ object LikeSimplification extends Rule[LogicalPlan] {
     }
   }
 
+  private def isSimplifyMultiLike(child: Expression): Boolean =
+    child.isInstanceOf[Attribute] || child.foldable

Review Comment:
   let's say if the data source can't push down string predicates, is it still useful to do multi-like optimization even if the input is an attribute?



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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1238121399

   > That commit is too risky. How about we just copy-paste the `isCheap` function when backporting this commit?
   
   OK.


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


[GitHub] [spark] wangyum commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
wangyum commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1229902140

   > what was the motivation to optimize multi-like before?
   
   It is mainly used for filtering pushdown.


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is complex expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957525963


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -771,6 +771,9 @@ object LikeSimplification extends Rule[LogicalPlan] {
     }
   }
 
+  private def isSimplifyMultiLike(child: Expression): Boolean =
+    child.isInstanceOf[Attribute] || child.foldable

Review Comment:
   OK, so multiple simple string operations is faster than a multi-like operation? Then I agree it's better to only optimize if input is a cheap expression. Can we reuse `CollapseProject.isCheap`?



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


[GitHub] [spark] wangyum commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not attribute

Posted by GitBox <gi...@apache.org>.
wangyum commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957357331


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -771,6 +771,9 @@ object LikeSimplification extends Rule[LogicalPlan] {
     }
   }
 
+  private def isSimplifyMultiLike(child: Expression): Boolean =
+    child.isInstanceOf[Attribute] || child.foldable

Review Comment:
   It may also be optimized to `StringStartsWith`, `StringEndsWith` and `StringContains`. For example:
   ```sql
   select * from t1 where id like all('%a', 'b%', '%c%', '%a%b%', '%a%b%c%')
   ```
   ```
   == Physical Plan ==
   *(1) Filter ((((isnotnull(id#7) AND EndsWith(id#7, a)) AND StartsWith(id#7, b)) AND Contains(id#7, c)) AND likeall(id#7, %a%b%, %a%b%c%))
   +- *(1) ColumnarToRow
      +- FileScan parquet spark_catalog.default.t1[id#7] Batched: true, DataFilters: [isnotnull(id#7), EndsWith(id#7, a), StartsWith(id#7, b), Contains(id#7, c), likeall(id#7, %a%b%,..., Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id), StringEndsWith(id,a), StringStartsWith(id,b), StringContains(id,c)], ReadSchema: struct<id:string>
   
   ```



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


[GitHub] [spark] wangyum commented on a diff in pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
wangyum commented on code in PR #37672:
URL: https://github.com/apache/spark/pull/37672#discussion_r957918402


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##########
@@ -771,6 +771,9 @@ object LikeSimplification extends Rule[LogicalPlan] {
     }
   }
 
+  private def isSimplifyMultiLike(child: Expression): Boolean =
+    child.isInstanceOf[Attribute] || child.foldable

Review Comment:
   OK



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


[GitHub] [spark] cloud-fan commented on pull request #37672: [SPARK-40228][SQL] Do not simplify multiLike if child is not a cheap expression

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on PR #37672:
URL: https://github.com/apache/spark/pull/37672#issuecomment-1237673481

   @wangyum shall we backport it all the way to 3.1?


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