-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25977][SQL] Parsing decimals from CSV using locale #22979
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 6 commits
c567dcc
2b41eba
cf438ae
f9438c4
3125c23
64a97a2
2f76352
3dfce18
bdca7c4
8c5593e
18470b0
c28b79f
1723da2
6cdafa5
14b5109
bab8fb2
5236336
0d1a4f0
8b1456c
0859624
e989b77
521bd45
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.csv | |
|
|
||
| import java.io.InputStream | ||
| import java.math.BigDecimal | ||
| import java.text.{DecimalFormat, DecimalFormatSymbols} | ||
|
|
||
| import scala.util.Try | ||
| import scala.util.control.NonFatal | ||
|
|
@@ -104,6 +105,12 @@ class UnivocityParser( | |
| requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray | ||
| } | ||
|
|
||
| private val decimalParser = { | ||
| val df = new DecimalFormat("", new DecimalFormatSymbols(options.locale)) | ||
MaxGekk marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| df.setParseBigDecimal(true) | ||
| df | ||
| } | ||
|
|
||
| /** | ||
| * Create a converter which converts the string value to a value according to a desired type. | ||
| * Currently, we do not support complex types (`ArrayType`, `MapType`, `StructType`). | ||
|
|
@@ -149,8 +156,8 @@ class UnivocityParser( | |
|
|
||
| case dt: DecimalType => (d: String) => | ||
| nullSafeDatum(d, name, nullable, options) { datum => | ||
| val value = new BigDecimal(datum.replaceAll(",", "")) | ||
| Decimal(value, dt.precision, dt.scale) | ||
| val bigDecimal = decimalParser.parse(datum).asInstanceOf[BigDecimal] | ||
|
||
| Decimal(bigDecimal, dt.precision, dt.scale) | ||
| } | ||
|
|
||
| case _: TimestampType => (d: String) => | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.