Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-42132

DeduplicateRelations rule breaks plan when co-grouping the same DataFrame

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.0.3, 3.1.3, 3.3.0, 3.3.1, 3.2.3, 3.4.0, 3.5.0
    • 3.5.0
    • SQL

    Description

      Co-grouping two DataFrames that share references breaks on the DeduplicateRelations rule:

      val df = spark.range(3)
      
      val left_grouped_df = df.groupBy("id").as[Long, Long]
      val right_grouped_df = df.groupBy("id").as[Long, Long]
      
      val cogroup_df = left_grouped_df.cogroup(right_grouped_df) {
        case (key, left, right) => left
      }
      
      cogroup_df.explain()
      
      == Physical Plan ==
      AdaptiveSparkPlan isFinalPlan=false
      +- SerializeFromObject [input[0, bigint, false] AS value#12L]
         +- CoGroup, id#0: bigint, id#0: bigint, id#0: bigint, [id#13L], [id#13L], [id#13L], [id#13L], obj#11: bigint
            :- !Sort [id#13L ASC NULLS FIRST], false, 0
            :  +- !Exchange hashpartitioning(id#13L, 200), ENSURE_REQUIREMENTS, [plan_id=16]
            :     +- Range (0, 3, step=1, splits=16)
            +- Sort [id#13L ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(id#13L, 200), ENSURE_REQUIREMENTS, [plan_id=17]
                  +- Range (0, 3, step=1, splits=16)
      

      The DataFrame cannot be computed:

      cogroup_df.show()
      
      java.lang.IllegalStateException: Couldn't find id#13L in [id#0L]
      

      The rule replaces `id#0L` on the right side with `id#13L` while replacing all occurrences in `CoGroup`. Some occurrences of `id#0L` in `CoGroup`refer to the left side and should not be replaced. Further, `id#0L` of the right deserializer is not replaced.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              enricomi Enrico Minack
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: