tgravescs commented on a change in pull request #30504:
URL: https://github.com/apache/spark/pull/30504#discussion_r531707223



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -873,24 +873,31 @@ object InferFiltersFromGenerate extends Rule[LogicalPlan] 
{
       if !e.deterministic || e.children.forall(_.foldable) => generate
 
     case generate @ Generate(g, _, false, _, _, _) if canInferFilters(g) =>
-      // Exclude child's constraints to guarantee idempotency
-      val inferredFilters = ExpressionSet(
-        Seq(
-          GreaterThan(Size(g.children.head), Literal(0)),
-          IsNotNull(g.children.head)
-        )
-      ) -- generate.child.constraints
-
-      if (inferredFilters.nonEmpty) {
-        generate.copy(child = Filter(inferredFilters.reduce(And), 
generate.child))
-      } else {
-        generate
+      assert(g.children.length == 1)
+      g.children.head match {
+        case _: CreateNonNullCollection =>

Review comment:
       I thought about using the nullable == false check  something like:
   
    ```
    private def canInferFilters(g: Generator): Boolean = g match {
       case cg: CollectionGenerator if (cg.children.head.nullable == true) => 
true
       case _ => false
     }
   ```
   
   but I think there could still be things we want the size == 0 check to apply 
to. For instance if doing explode(col2) where col2 contained an empty Map or 
Array. I think this case you still want the size == 0 check. Please correct me 
if I'm missing something.




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

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