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

Should not coerce decimal type to double type when it's join column

    XMLWordPrintableJSON

Details

    • Bug
    • Status: In Progress
    • Major
    • Resolution: Unresolved
    • 2.3.4, 2.4.4, 3.0.0
    • None
    • SQL
    • None

    Description

      How to reproduce this issue:

      create table t1 (incdata_id decimal(21,0), v string) using parquet;
      create table t2 (incdata_id string, v string) using parquet;
      
      explain select * from t1 join t2 on (t1.incdata_id = t2.incdata_id);
      == Physical Plan ==
      *(5) SortMergeJoin [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31 as double)))], [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33 as double)))], Inner
      :- *(2) Sort [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31 as double))) ASC NULLS FIRST], false, 0
      :  +- Exchange hashpartitioning(knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#31 as double))), 200), true, [id=#104]
      :     +- *(1) Filter isnotnull(incdata_id#31)
      :        +- Scan hive default.t1 [incdata_id#31, v#32], HiveTableRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [incdata_id#31, v#32], Statistics(sizeInBytes=8.0 EiB)
      +- *(4) Sort [knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33 as double))) ASC NULLS FIRST], false, 0
         +- Exchange hashpartitioning(knownfloatingpointnormalized(normalizenanandzero(cast(incdata_id#33 as double))), 200), true, [id=#112]
            +- *(3) Filter isnotnull(incdata_id#33)
               +- Scan hive default.t2 [incdata_id#33, v#34], HiveTableRelation `default`.`t2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [incdata_id#33, v#34], Statistics(sizeInBytes=8.0 EiB)
      
      select cast(v1 as double) as v3, cast(v2 as double) as v4,
        cast(v1 as double) = cast(v2 as double), v1 = v2 
      from (select cast('100000000001636981212' as decimal(21, 0)) as v1,
            cast('100000000001636981213' as decimal(21, 0)) as v2) t;
      
      1.0000000000163697E20	1.0000000000163697E20	true	false
      

       

      It's a realy case in our production:

      Attachments

        1. image-2019-09-27-20-20-24-238.png
          4.36 MB
          Yuming Wang

        Issue Links

          Activity

            People

              Unassigned Unassigned
              yumwang Yuming Wang
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated: