You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Anton Okolnychyi (JIRA)" <ji...@apache.org> on 2017/07/25 20:14:00 UTC

[jira] [Commented] (SPARK-21479) Outer join filter pushdown in null supplying table when condition is on one of the joined columns

    [ https://issues.apache.org/jira/browse/SPARK-21479?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16100691#comment-16100691 ] 

Anton Okolnychyi commented on SPARK-21479:
------------------------------------------

I used the following code to investigate:

{code}
    val inputSchema1 = StructType(
      StructField("col1", StringType) ::
      StructField("col2", IntegerType) ::
      Nil)

    val inputSchema2 = StructType(
      StructField("col1", StringType) ::
      StructField("col3", StringType) ::
      Nil)

    val rdd1 = sc.parallelize(1 to 3).map(v => Row(s"value $v", v))
    val df1 = spark.createDataFrame(rdd1, inputSchema1)
    val rdd2 = sc.parallelize(1 to 3).map(v => Row(s"value $v", "some value"))
    val df2 = spark.createDataFrame(rdd2, inputSchema2)
    
    // 1st use case
    df1.join(df2, Seq("col1"), "right_outer").where("col2 = 2").explain(true)
    // 2nd use case
    df1.join(df2, Seq("col1"), "right_outer").where("col1 = 'value 2'").explain(true)
{code}

It is important to notice that the actual join type in the first case is `inner` and not `rigth_outer`. This happens due to the `EliminateOuterJoin` rule, which sees that `col2 = 2` filters out non-matching rows on the left side of the join. Once the join type is changed, the `PushPredicateThroughJoin` rule pushes `col2 = 2` to the left relation. The analyzed and optimized logical plans are:

{noformat}
== Analyzed Logical Plan ==
col1: string, col2: int, col3: string
Filter (col2#3 = 2)
+- Project [col1#9, col2#3, col3#10]
   +- Join RightOuter, (col1#2 = col1#9)
      :- LogicalRDD [col1#2, col2#3]
      +- LogicalRDD [col1#9, col3#10]

== Optimized Logical Plan ==
Project [col1#9, col2#3, col3#10]
+- Join Inner, (col1#2 = col1#9)
   :- Filter ((isnotnull(col2#3) && (col2#3 = 2)) && isnotnull(col1#2))
   :  +- LogicalRDD [col1#2, col2#3]
   +- Filter isnotnull(col1#9)
      +- LogicalRDD [col1#9, col3#10]

{noformat}

The second case is different. The join type stays the same (i.e., `right_outer`) and the analyzed logical plan looks like:

{noformat}
== Analyzed Logical Plan ==
col1: string, col2: int, col3: string
Filter (col1#9 = value 2)
+- Project [col1#9, col2#3, col3#10]
   +- Join RightOuter, (col1#2 = col1#9)
      :- LogicalRDD [col1#2, col2#3]
      +- LogicalRDD [col1#9, col3#10]
{noformat}

`col1#9` from the Filter belongs to the right relation. After `PushPredicateThroughJoin` we have:

{noformat}
Join RightOuter, (col1#2 = col1#9)
:- LogicalRDD [col1#2, col2#3]
+- Filter (isnotnull(col1#9) && (col1#9 = value 2))
   +- LogicalRDD [col1#9, col3#10]
{noformat}


In theory, `InferFiltersFromConstraints` is capable of inferring `(col1#2 = value 2)` from `(col1#9 = value 2, col1#2 = col1#9)`. However, not in this case since the join type is `right_outer` and `InferFiltersFromConstraints` will process only constraints from the right relation (i.e., `(isnotnull(col1#9) && (col1#9 = value 2))`), which is not enough to infer `(col1#2 = value 2)`.

It seems like this is done on purpose and it is expected behavior even though additional `(col1#2 = value 2)` on the left relation would be logically correct here (as far as I understand).

> Outer join filter pushdown in null supplying table when condition is on one of the joined columns
> -------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-21479
>                 URL: https://issues.apache.org/jira/browse/SPARK-21479
>             Project: Spark
>          Issue Type: Bug
>          Components: Optimizer, SQL
>    Affects Versions: 2.1.0, 2.1.1, 2.2.0
>            Reporter: Abhijit Bhole
>
> Here are two different query plans - 
> {code:java}
> df1 = spark.createDataFrame([{ "a": 1, "b" : 2}, { "a": 3, "b" : 4}])
> df2 = spark.createDataFrame([{ "a": 1, "c" : 5}, { "a": 3, "c" : 6}, { "a": 5, "c" : 8}])
> df1.join(df2, ['a'], 'right_outer').where("b = 2").explain()
> == Physical Plan ==
> *Project [a#16299L, b#16295L, c#16300L]
> +- *SortMergeJoin [a#16294L], [a#16299L], Inner
>    :- *Sort [a#16294L ASC NULLS FIRST], false, 0
>    :  +- Exchange hashpartitioning(a#16294L, 4)
>    :     +- *Filter ((isnotnull(b#16295L) && (b#16295L = 2)) && isnotnull(a#16294L))
>    :        +- Scan ExistingRDD[a#16294L,b#16295L]
>    +- *Sort [a#16299L ASC NULLS FIRST], false, 0
>       +- Exchange hashpartitioning(a#16299L, 4)
>          +- *Filter isnotnull(a#16299L)
>             +- Scan ExistingRDD[a#16299L,c#16300L]
> df1 = spark.createDataFrame([{ "a": 1, "b" : 2}, { "a": 3, "b" : 4}])
> df2 = spark.createDataFrame([{ "a": 1, "c" : 5}, { "a": 3, "c" : 6}, { "a": 5, "c" : 8}])
> df1.join(df2, ['a'], 'right_outer').where("a = 1").explain()
> == Physical Plan ==
> *Project [a#16314L, b#16310L, c#16315L]
> +- SortMergeJoin [a#16309L], [a#16314L], RightOuter
>    :- *Sort [a#16309L ASC NULLS FIRST], false, 0
>    :  +- Exchange hashpartitioning(a#16309L, 4)
>    :     +- Scan ExistingRDD[a#16309L,b#16310L]
>    +- *Sort [a#16314L ASC NULLS FIRST], false, 0
>       +- Exchange hashpartitioning(a#16314L, 4)
>          +- *Filter (isnotnull(a#16314L) && (a#16314L = 1))
>             +- Scan ExistingRDD[a#16314L,c#16315L]
> {code}
> If condition on b can be pushed down on df1 then why not condition on a?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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