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

Join on distinct column with monotonically_increasing_id produces wrong output

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.0.2, 2.1.3, 2.2.2, 2.3.2, 2.4.0
    • 2.3.4, 2.4.1, 3.0.0
    • SQL
    • Running on Ubuntu 18.04LTS and Intellij 2018.2.5

    Description

      When joining a table with projected monotonically_increasing_id column after calling distinct with another table the operators do not get executed in the right order. 

      Here is a minimal example:

      import org.apache.spark.sql.{DataFrame, SparkSession, functions}
      
      object JoinBug extends App {
      
        // Spark session setup
        val session =  SparkSession.builder().master("local[*]").getOrCreate()
        import session.sqlContext.implicits._
        session.sparkContext.setLogLevel("error")
      
        // Bug in Spark: "monotonically_increasing_id" is pushed down when it shouldn't be. Push down only happens when the
        // DF containing the "monotonically_increasing_id" expression is on the left side of the join.
        val baseTable = Seq((1), (1)).toDF("idx")
        val distinctWithId = baseTable.distinct.withColumn("id", functions.monotonically_increasing_id())
        val monotonicallyOnRight: DataFrame = baseTable.join(distinctWithId, "idx")
        val monotonicallyOnLeft: DataFrame = distinctWithId.join(baseTable, "idx")
      
        monotonicallyOnLeft.show // Wrong
        monotonicallyOnRight.show // Ok in Spark 2.2.2 - also wrong in Spark 2.4.0
      
      }
      
      

      It produces the following output:

      Wrong:
      +---+------------+
      |idx| id         |
      +---+------------+
      | 1|369367187456 |
      | 1|369367187457 |
      +---+------------+
      
      Right:
      +---+------------+
      |idx| id         |
      +---+------------+
      | 1|369367187456 |
      | 1|369367187456 |
      +---+------------+
      

      We assume that the join operator triggers a pushdown of expressions (monotonically_increasing_id in this case) which gets pushed down to be executed before distinct. This produces non-distinct rows with unique id's. However it seems like this behavior only appears if the table with the projected expression is on the left side of the join in Spark 2.2.2 (for version 2.4.0 it fails on both joins).

      Attachments

        Issue Links

          Activity

            People

              petertoth Peter Toth
              soerenreichardt Sören Reichardt
              Votes:
              2 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: