Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-6116 DataFrame API improvement umbrella ticket (Spark 1.5)
  3. SPARK-6231

Join on two tables (generated from same one) is broken

    XMLWordPrintableJSON

    Details

    • Type: Sub-task
    • Status: Resolved
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 1.3.0
    • Fix Version/s: 1.4.0
    • Component/s: SQL
    • Labels:
    • Target Version/s:
    • Sprint:
      Spark 1.5 doc/QA sprint

      Description

      If the two column used in joinExpr come from the same table, they have the same id, then the joniExpr is explained in wrong way.

      val df = sqlContext.load(path, "parquet")
      
      val txns = df.groupBy("cust_id").agg($"cust_id", countDistinct($"day_num").as("txns"))
      
      val spend = df.groupBy("cust_id").agg($"cust_id", sum($"extended_price").as("spend"))
      
      val rmJoin = txns.join(spend, txns("cust_id") === spend("cust_id"), "inner")
      
      scala> rmJoin.explain
      == Physical Plan ==
      CartesianProduct
       Filter (cust_id#0 = cust_id#0)
        Aggregate false, [cust_id#0], [cust_id#0,CombineAndCount(partialSets#25) AS txns#7L]
         Exchange (HashPartitioning [cust_id#0], 200)
          Aggregate true, [cust_id#0], [cust_id#0,AddToHashSet(day_num#2L) AS partialSets#25]
           PhysicalRDD [cust_id#0,day_num#2L], MapPartitionsRDD[1] at map at newParquet.scala:542
       Aggregate false, [cust_id#17], [cust_id#17,SUM(PartialSum#38) AS spend#8]
        Exchange (HashPartitioning [cust_id#17], 200)
         Aggregate true, [cust_id#17], [cust_id#17,SUM(extended_price#20) AS PartialSum#38]
          PhysicalRDD [cust_id#17,extended_price#20], MapPartitionsRDD[3] at map at newParquet.scala:542
      
      

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                rxin Reynold Xin
                Reporter:
                davies Davies Liu
              • Votes:
                1 Vote for this issue
                Watchers:
                12 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: