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

Incorrect result when work with data from parquet

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Blocker
    • Resolution: Fixed
    • Affects Version/s: 2.0.0, 2.0.1
    • Fix Version/s: 2.0.2, 2.1.0
    • Component/s: SQL
    • Labels:

      Description

        import org.apache.spark.SparkConf
        import org.apache.spark.sql.SparkSession
        import org.apache.spark.sql.types.{StructField, StructType}
        import org.apache.spark.sql.types.DataTypes._
      
        val sc = SparkSession.builder().config(new SparkConf().setMaster("local")).getOrCreate()
      
        val jsonRDD = sc.sparkContext.parallelize(Seq(
          """{"a":1,"b":1,"c":1}""",
          """{"a":1,"b":1,"c":2}"""
        ))
      
        sc.read.schema(StructType(Seq(
          StructField("a", IntegerType),
          StructField("b", IntegerType),
          StructField("c", LongType)
        ))).json(jsonRDD).write.parquet("/tmp/test")
      
        val df = sc.read.load("/tmp/test")
        df.join(df, Seq("a", "b", "c"), "left_outer").show()
      

      returns:

      +---+---+---+
      |  a|  b|  c|
      +---+---+---+
      |  1|  1|  1|
      |  1|  1|  1|
      |  1|  1|  2|
      |  1|  1|  2|
      +---+---+---+
      

      Expected result:

      +---+---+---+
      |  a|  b|  c|
      +---+---+---+
      |  1|  1|  1|
      |  1|  1|  2|
      +---+---+---+
      

      If I use this code without saving to parquet it works fine. If you change type of `c` column to `IntegerType` it also works fine.

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                davies Davies Liu
                Reporter:
                v-gerasimov Vitaly Gerasimov
              • Votes:
                0 Vote for this issue
                Watchers:
                6 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: