You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Apache Spark (JIRA)" <ji...@apache.org> on 2018/12/13 11:45:00 UTC

[jira] [Assigned] (SPARK-26308) Large BigDecimal value is converted to null when passed into a UDF

     [ https://issues.apache.org/jira/browse/SPARK-26308?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Apache Spark reassigned SPARK-26308:
------------------------------------

    Assignee: Apache Spark

> Large BigDecimal value is converted to null when passed into a UDF
> ------------------------------------------------------------------
>
>                 Key: SPARK-26308
>                 URL: https://issues.apache.org/jira/browse/SPARK-26308
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.0
>            Reporter: Jay Pranavamurthi
>            Assignee: Apache Spark
>            Priority: Major
>
> We are loading a Hive table into a Spark DataFrame. The Hive table has a decimal(30, 0) column with values greater than Long.MAX_VALUE. The DataFrame loads correctly.
> We then use a UDF to convert the decimal type to a String value. For decimal values < Long.MAX_VALUE, this works fine, but when the decimal value > Long.MAX_VALUE, the input to the UDF is a *null*.
> Hive table schema and data:
> {code:java}
> create table decimal_test (col1 decimal(30, 0), col2 decimal(10, 0), col3 int, col4 string);
> insert into decimal_test values(2011000000000002456556, 123456789, 10, 'test1');
> {code}
>  
> Execution in spark-shell:
> _(Note that the first column in the final output is null, it should have been "2011000000000002456556")_
> {code:java}
> scala> val df1 = spark.sqlContext.sql("select * from decimal_test")
> df1: org.apache.spark.sql.DataFrame = [col1: decimal(30,0), col2: decimal(10,0) ... 2 more fields]
> scala> df1.show
> +--------------------+---------+----+-----+
> | col1| col2|col3| col4|
> +--------------------+---------+----+-----+
> |20110000000000024...|123456789| 10|test1|
> +--------------------+---------+----+-----+
> scala> val decimalToString = (value: java.math.BigDecimal) => if (value == null) null else { value.toBigInteger().toString }
> decimalToString: java.math.BigDecimal => String = <function1>
> scala> val udf1 = org.apache.spark.sql.functions.udf(decimalToString)
> udf1: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedFunction(<function1>,StringType,Some(List(DecimalType(38,18))))
> scala> val df2 = df1.withColumn("col1", udf1(df1.col("col1")))
> df2: org.apache.spark.sql.DataFrame = [col1: string, col2: decimal(10,0) ... 2 more fields]
> scala> df2.show
> +----+---------+----+-----+
> |col1| col2|col3| col4|
> +----+---------+----+-----+
> |null|123456789| 10|test1|
> +----+---------+----+-----+
> {code}
> Oddly this works if we change the "decimalToString" udf to take an "Any" instead of a "java.math.BigDecimal"
> {code:java}
> scala> val decimalToString = (value: Any) => if (value == null) null else { if (value.isInstanceOf[java.math.BigDecimal]) value.asInstanceOf[java.math.BigDecimal].toBigInteger().toString else null }
> decimalToString: Any => String = <function1>
> scala> val udf1 = org.apache.spark.sql.functions.udf(decimalToString)
> udf1: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedFunction(<function1>,StringType,None)
> scala> val df2 = df1.withColumn("col1", udf1(df1.col("col1")))
> df2: org.apache.spark.sql.DataFrame = [col1: string, col2: decimal(10,0) ... 2 more fields]
> scala> df2.show
> +--------------------+---------+----+-----+
> | col1| col2|col3| col4|
> +--------------------+---------+----+-----+
> |20110000000000024...|123456789| 10|test1|
> +--------------------+---------+----+-----+
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org