aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala27
2 files changed, 28 insertions, 1 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
index 99ac7730bd..330ba907b2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
@@ -95,7 +95,7 @@ private[sql] object JacksonGenerator {
case (FloatType, v: Float) => gen.writeNumber(v)
case (DoubleType, v: Double) => gen.writeNumber(v)
case (LongType, v: Long) => gen.writeNumber(v)
- case (DecimalType(), v: java.math.BigDecimal) => gen.writeNumber(v)
+ case (DecimalType(), v: Decimal) => gen.writeNumber(v.toJavaBigDecimal)
case (ByteType, v: Byte) => gen.writeNumber(v.toInt)
case (BinaryType, v: Array[Byte]) => gen.writeBinary(v)
case (BooleanType, v: Boolean) => gen.writeBoolean(v)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
index ed6d512ab3..8ca3a17085 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.sources
+import java.math.BigDecimal
+
import org.apache.hadoop.fs.Path
import org.apache.spark.deploy.SparkHadoopUtil
@@ -75,4 +77,29 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest {
)
}
}
+
+ test("SPARK-10196: save decimal type to JSON") {
+ withTempDir { file =>
+ file.delete()
+
+ val schema =
+ new StructType()
+ .add("decimal", DecimalType(7, 2))
+
+ val data =
+ Row(new BigDecimal("10.02")) ::
+ Row(new BigDecimal("20000.99")) ::
+ Row(new BigDecimal("10000")) :: Nil
+ val df = createDataFrame(sparkContext.parallelize(data), schema)
+
+ // Write the data out.
+ df.write.format(dataSourceName).save(file.getCanonicalPath)
+
+ // Read it back and check the result.
+ checkAnswer(
+ read.format(dataSourceName).schema(schema).load(file.getCanonicalPath),
+ df
+ )
+ }
+ }
}