## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-14231
Currently, JSON data source supports to infer `DecimalType` for big numbers and `floatAsBigDecimal` option which reads floating-point values as `DecimalType`.
But there are few restrictions in Spark `DecimalType` below:
1. The precision cannot be bigger than 38.
2. scale cannot be bigger than precision.
Currently, both restrictions are not being handled.
This PR handles the cases by inferring them as `DoubleType`. Also, the option name was changed from `floatAsBigDecimal` to `prefersDecimal` as suggested [here](https://issues.apache.org/jira/browse/SPARK-14231?focusedCommentId=15215579&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15215579).
So, the codes below:
```scala
def doubleRecords: RDD[String] =
sqlContext.sparkContext.parallelize(
s"""{"a": 1${"0" * 38}, "b": 0.01}""" ::
s"""{"a": 2${"0" * 38}, "b": 0.02}""" :: Nil)
val jsonDF = sqlContext.read
.option("prefersDecimal", "true")
.json(doubleRecords)
jsonDF.printSchema()
```
produces below:
- **Before**
```scala
org.apache.spark.sql.AnalysisException: Decimal scale (2) cannot be greater than precision (1).;
at org.apache.spark.sql.types.DecimalType.<init>(DecimalType.scala:44)
at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:144)
at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:108)
at
...
```
- **After**
```scala
root
|-- a: double (nullable = true)
|-- b: double (nullable = true)
```
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for coding style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#12030 from HyukjinKwon/SPARK-14231.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13953
Currently, JSON data source creates a new field in `PERMISSIVE` mode for storing malformed string.
This field can be renamed via `spark.sql.columnNameOfCorruptRecord` option but it is a global configuration.
This PR make that option can be applied per read and can be specified via `option()`. This will overwrites `spark.sql.columnNameOfCorruptRecord` if it is set.
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for coding style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11881 from HyukjinKwon/SPARK-13953.
## What changes were proposed in this pull request?
Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .
This PR adds the support for parse modes just like CSV data source. There are three modes below:
- `PERMISSIVE` : When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.
This PR also make JSON data source share the `ParseModes` in CSV data source.
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for code style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11756 from HyukjinKwon/SPARK-13764.
## What changes were proposed in this pull request?
This PR adds the support to specify compression codecs for both ORC and Parquet.
## How was this patch tested?
unittests within IDE and code style tests with `dev/run_tests`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11464 from HyukjinKwon/SPARK-13543.
https://issues.apache.org/jira/browse/SPARK-13507https://issues.apache.org/jira/browse/SPARK-13509
## What changes were proposed in this pull request?
This PR adds the support to write CSV data directly by a single call to the given path.
Several unitests were added for each functionality.
## How was this patch tested?
This was tested with unittests and with `dev/run_tests` for coding style
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Closes#11389 from HyukjinKwon/SPARK-13507-13509.
I tried to add this via `USE_BIG_DECIMAL_FOR_FLOATS` option from Jackson with no success.
Added test for non-complex types. Should I add a test for complex types?
Author: Brandon Bradley <bradleytastic@gmail.com>
Closes#10936 from blbradley/spark-12749.
We can provides the option to choose JSON parser can be enabled to accept quoting of all character or not.
Author: Cazen <Cazen@korea.com>
Author: Cazen Lee <cazen.lee@samsung.com>
Author: Cazen Lee <Cazen@korea.com>
Author: cazen.lee <cazen.lee@samsung.com>
Closes#10497 from Cazen/master.
Since we rename the column name from ```text``` to ```value``` for DataFrame load by ```SQLContext.read.text```, we need to update doc.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#10349 from yanboliang/text-value.
This patch makes it consistent to use varargs in all DataFrameReader methods, including Parquet, JSON, text, and the generic load function.
Also added a few more API tests for the Java API.
Author: Reynold Xin <rxin@databricks.com>
Closes#9945 from rxin/SPARK-11967.
This patch adds the following options to the JSON data source, for dealing with non-standard JSON files:
* `allowComments` (default `false`): ignores Java/C++ style comment in JSON records
* `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names
* `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes
* `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers (e.g. 00012)
To avoid passing a lot of options throughout the json package, I introduced a new JSONOptions case class to define all JSON config options.
Also updated documentation to explain these options.
Scala
![screen shot 2015-11-15 at 6 12 12 pm](https://cloud.githubusercontent.com/assets/323388/11172965/e3ace6ec-8bc4-11e5-805e-2d78f80d0ed6.png)
Python
![screen shot 2015-11-15 at 6 11 28 pm](https://cloud.githubusercontent.com/assets/323388/11172964/e23ed6ee-8bc4-11e5-8216-312f5983acd5.png)
Author: Reynold Xin <rxin@databricks.com>
Closes#9724 from rxin/SPARK-11745.
Make sure comma-separated paths get processed correcly in ResolvedDataSource for a HadoopFsRelationProvider
Author: Koert Kuipers <koert@tresata.com>
Closes#8416 from koertkuipers/feat-sql-comma-separated-paths.
PySpark DataFrameReader should could accept an RDD of Strings (like the Scala version does) for JSON, rather than only taking a path.
If this PR is merged, it should be duplicated to cover the other input types (not just JSON).
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#8444 from yanboliang/spark-9964.
This PR adds DataFrame reader/writer shortcut methods for ORC in both Scala and Python.
Author: Cheng Lian <lian@databricks.com>
Closes#7444 from liancheng/spark-9100 and squashes the following commits:
284d043 [Cheng Lian] Fixes PySpark test cases and addresses PR comments
e0b09fb [Cheng Lian] Adds DataFrame reader/writer shortcut methods for ORC
Author: Reynold Xin <rxin@databricks.com>
Closes#7079 from rxin/SPARK-8698 and squashes the following commits:
8513e1c [Reynold Xin] [SPARK-8698] partitionBy in Python DataFrame reader/writer interface should not default to empty tuple.
I compared PySpark DataFrameReader/Writer against Scala ones. `Option` function is missing in both reader and writer, but the rest seems to all match.
I added `Option` to reader and writer and updated the `pyspark-sql` test.
Author: Cheolsoo Park <cheolsoop@netflix.com>
Closes#7078 from piaozhexiu/SPARK-8355 and squashes the following commits:
c63d419 [Cheolsoo Park] Fix version
524e0aa [Cheolsoo Park] Add option function to df reader and writer
https://issues.apache.org/jira/browse/SPARK-8532
This PR has two changes. First, it fixes the bug that save actions (i.e. `save/saveAsTable/json/parquet/jdbc`) always override mode. Second, it adds input argument `partitionBy` to `save/saveAsTable/parquet`.
Author: Yin Huai <yhuai@databricks.com>
Closes#6937 from yhuai/SPARK-8532 and squashes the following commits:
f972d5d [Yin Huai] davies's comment.
d37abd2 [Yin Huai] style.
d21290a [Yin Huai] Python doc.
889eb25 [Yin Huai] Minor refactoring and add partitionBy to save, saveAsTable, and parquet.
7fbc24b [Yin Huai] Use None instead of "error" as the default value of mode since JVM-side already uses "error" as the default value.
d696dff [Yin Huai] Python style.
88eb6c4 [Yin Huai] If mode is "error", do not call mode method.
c40c461 [Yin Huai] Regression test.
add schema()/format()/options() for reader, add mode()/format()/options()/partitionBy() for writer
cc rxin yhuai pwendell
Author: Davies Liu <davies@databricks.com>
Closes#6578 from davies/readwrite and squashes the following commits:
720d293 [Davies Liu] address comments
b65dfa2 [Davies Liu] Update readwriter.py
1299ab6 [Davies Liu] make Python API consistent with Scala
Add tests later.
Author: Davies Liu <davies@databricks.com>
Closes#6375 from davies/insertInto and squashes the following commits:
826423e [Davies Liu] add insertInto() to Writer
Add version info for public Python SQL API.
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#6295 from davies/versions and squashes the following commits:
cfd91e6 [Davies Liu] add more version for DataFrame API
600834d [Davies Liu] add version to SQL API docs
cc rxin, please take a quick look, I'm working on tests.
Author: Davies Liu <davies@databricks.com>
Closes#6238 from davies/readwrite and squashes the following commits:
c7200eb [Davies Liu] update tests
9cbf01b [Davies Liu] Merge branch 'master' of github.com:apache/spark into readwrite
f0c5a04 [Davies Liu] use sqlContext.read.load
5f68bc8 [Davies Liu] update tests
6437e9a [Davies Liu] Merge branch 'master' of github.com:apache/spark into readwrite
bcc6668 [Davies Liu] add reader amd writer API in Python