You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Takeshi Yamamuro (JIRA)" <ji...@apache.org> on 2018/04/25 04:26:00 UTC

[jira] [Created] (SPARK-24079) Update the nullability of Join output based on inferred predicates

Takeshi Yamamuro created SPARK-24079:
----------------------------------------

             Summary: Update the nullability of Join output based on inferred predicates
                 Key: SPARK-24079
                 URL: https://issues.apache.org/jira/browse/SPARK-24079
             Project: Spark
          Issue Type: Improvement
          Components: SQL
    Affects Versions: 2.3.0
            Reporter: Takeshi Yamamuro



In the master, a logical `Join` node does not respect the nullability that the optimizer rule `InferFiltersFromConstraints`
might change when inferred predicates have `IsNotNull`, e.g.,

{code}
scala> val df1 = Seq((Some(1), Some(2))).toDF("k", "v0")
scala> val df2 = Seq((Some(1), Some(3))).toDF("k", "v1")
scala> val joinedDf = df1.join(df2, df1("k") === df2("k"), "inner")
scala> joinedDf.explain
== Physical Plan ==
*(2) BroadcastHashJoin [k#83], [k#92], Inner, BuildRight
:- *(2) Project [_1#80 AS k#83, _2#81 AS v0#84]
:  +- *(2) Filter isnotnull(_1#80)
:     +- LocalTableScan [_1#80, _2#81]
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)))
   +- *(1) Project [_1#89 AS k#92, _2#90 AS v1#93]
      +- *(1) Filter isnotnull(_1#89)
         +- LocalTableScan [_1#89, _2#90]

scala> joinedDf.queryExecution.optimizedPlan.output.map(_.nullable)
res15: Seq[Boolean] = List(true, true, true, true)
{code}

But, these `nullable` values should be:

{code}
scala> joinedDf.queryExecution.optimizedPlan.output.map(_.nullable)
res15: Seq[Boolean] = List(false, true, false, true)
{code}

This ticket comes from the previous discussion: https://github.com/apache/spark/pull/18576#pullrequestreview-107585997




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org