-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-15982][SPARK-16009][SPARK-16007][SQL] Harmonize the behavior of DataFrameReader.text/csv/json/parquet/orc #13727
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 5 commits
dcc4655
960048d
3384473
3150b01
bb52410
29524b1
24174f0
3498bd0
2539a94
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 |
|---|---|---|
|
|
@@ -119,13 +119,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| * @since 1.4.0 | ||
| */ | ||
| def load(): DataFrame = { | ||
| val dataSource = | ||
| DataSource( | ||
| sparkSession, | ||
| userSpecifiedSchema = userSpecifiedSchema, | ||
| className = source, | ||
| options = extraOptions.toMap) | ||
| Dataset.ofRows(sparkSession, LogicalRelation(dataSource.resolveRelation())) | ||
| load(Seq.empty: _*) // force invocation of `load(...varargs...)` | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -135,7 +129,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| * @since 1.4.0 | ||
| */ | ||
| def load(path: String): DataFrame = { | ||
| option("path", path).load() | ||
| load(Seq(path): _*) // force invocation of `load(...varargs...)` | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. with this change path is no longer available in the options. this makes it hard (impossible?) for non-file based DataSources (not implementing FileFormat) to use load(...) For example for elasticsearch we use: i do not think this can be implemented anymore now?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sqlContext.read.option("path", resource).format("org.elasticsearch.spark.sql").load()Can you try this?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i believe that works as expected (i am running into some other issues now, but they seem unrelated).
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it will also break users code in an upgrade
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yea this is a bad breaking change.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you want me to fix it?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If you can!
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure, will do it soon. Thanks! |
||
| } | ||
|
|
||
| /** | ||
|
|
@@ -146,18 +140,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| */ | ||
| @scala.annotation.varargs | ||
| def load(paths: String*): DataFrame = { | ||
| if (paths.isEmpty) { | ||
| sparkSession.emptyDataFrame | ||
| } else { | ||
| sparkSession.baseRelationToDataFrame( | ||
| DataSource.apply( | ||
| sparkSession, | ||
| paths = paths, | ||
| userSpecifiedSchema = userSpecifiedSchema, | ||
| className = source, | ||
| options = extraOptions.toMap).resolveRelation()) | ||
| } | ||
| sparkSession.baseRelationToDataFrame( | ||
| DataSource.apply( | ||
| sparkSession, | ||
| paths = paths, | ||
| userSpecifiedSchema = userSpecifiedSchema, | ||
| className = source, | ||
| options = extraOptions.toMap).resolveRelation()) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. removed the special handling of empty paths
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In my PR, will add the test cases to verify all the possible inputs after this code changes. Thanks! |
||
| } | ||
|
|
||
| /** | ||
| * Construct a [[DataFrame]] representing the database table accessible via JDBC URL | ||
| * url named table and connection properties. | ||
|
|
@@ -276,7 +267,45 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string | ||
| * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.</li> | ||
| * | ||
| * @since 1.6.0 | ||
| * @since 1.4.0 | ||
| */ | ||
| def json(path: String): DataFrame = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| json(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads a JSON file (one object per line) and returns the result as a [[DataFrame]]. | ||
| * | ||
| * This function goes through the input once to determine the input schema. If you know the | ||
| * schema in advance, use the version that specifies the schema to avoid the extra scan. | ||
| * | ||
| * You can set the following JSON-specific options to deal with non-standard JSON files: | ||
| * <li>`primitivesAsString` (default `false`): infers all primitive values as a string type</li> | ||
| * <li>`prefersDecimal` (default `false`): infers all floating-point values as a decimal | ||
| * type. If the values do not fit in decimal, then it infers them as doubles.</li> | ||
| * <li>`allowComments` (default `false`): ignores Java/C++ style comment in JSON records</li> | ||
| * <li>`allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names</li> | ||
| * <li>`allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes | ||
| * </li> | ||
| * <li>`allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers | ||
| * (e.g. 00012)</li> | ||
| * <li>`allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all | ||
| * character using backslash quoting mechanism</li> | ||
| * <li>`mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records | ||
| * during parsing.</li> | ||
| * <ul> | ||
| * <li>`PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the | ||
|
||
| * malformed string into a new field configured by `columnNameOfCorruptRecord`. When | ||
| * a schema is set by user, it sets `null` for extra fields.</li> | ||
| * <li>`DROPMALFORMED` : ignores the whole corrupted records.</li> | ||
| * <li>`FAILFAST` : throws an exception when it meets corrupted records.</li> | ||
| * </ul> | ||
| * <li>`columnNameOfCorruptRecord` (default is the value specified in | ||
| * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string | ||
| * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.</li> | ||
| * | ||
| * @since 2.0.0 | ||
| */ | ||
| @scala.annotation.varargs | ||
| def json(paths: String*): DataFrame = format("json").load(paths : _*) | ||
|
|
@@ -326,6 +355,63 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| parsedOptions))(sparkSession)) | ||
| } | ||
|
|
||
| /** | ||
| * Loads a CSV file and returns the result as a [[DataFrame]]. | ||
| * | ||
| * This function will go through the input once to determine the input schema if `inferSchema` | ||
| * is enabled. To avoid going through the entire data once, disable `inferSchema` option or | ||
| * specify the schema explicitly using [[schema]]. | ||
| * | ||
| * You can set the following CSV-specific options to deal with CSV files: | ||
| * <li>`sep` (default `,`): sets the single character as a separator for each | ||
| * field and value.</li> | ||
| * <li>`encoding` (default `UTF-8`): decodes the CSV files by the given encoding | ||
| * type.</li> | ||
| * <li>`quote` (default `"`): sets the single character used for escaping quoted values where | ||
| * the separator can be part of the value. If you would like to turn off quotations, you need to | ||
| * set not `null` but an empty string. This behaviour is different form | ||
| * `com.databricks.spark.csv`.</li> | ||
| * <li>`escape` (default `\`): sets the single character used for escaping quotes inside | ||
| * an already quoted value.</li> | ||
| * <li>`comment` (default empty string): sets the single character used for skipping lines | ||
| * beginning with this character. By default, it is disabled.</li> | ||
| * <li>`header` (default `false`): uses the first line as names of columns.</li> | ||
| * <li>`inferSchema` (default `false`): infers the input schema automatically from data. It | ||
| * requires one extra pass over the data.</li> | ||
| * <li>`ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces | ||
| * from values being read should be skipped.</li> | ||
| * <li>`ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing | ||
| * whitespaces from values being read should be skipped.</li> | ||
| * <li>`nullValue` (default empty string): sets the string representation of a null value.</li> | ||
| * <li>`nanValue` (default `NaN`): sets the string representation of a non-number" value.</li> | ||
| * <li>`positiveInf` (default `Inf`): sets the string representation of a positive infinity | ||
| * value.</li> | ||
| * <li>`negativeInf` (default `-Inf`): sets the string representation of a negative infinity | ||
| * value.</li> | ||
| * <li>`dateFormat` (default `null`): sets the string that indicates a date format. Custom date | ||
| * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type | ||
| * and timestamp type. By default, it is `null` which means trying to parse times and date by | ||
| * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.</li> | ||
| * <li>`maxColumns` (default `20480`): defines a hard limit of how many columns | ||
| * a record can have.</li> | ||
| * <li>`maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed | ||
| * for any given value being read.</li> | ||
| * <li>`mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records | ||
| * during parsing.</li> | ||
| * <ul> | ||
| * <li>`PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When | ||
| * a schema is set by user, it sets `null` for extra fields.</li> | ||
| * <li>`DROPMALFORMED` : ignores the whole corrupted records.</li> | ||
| * <li>`FAILFAST` : throws an exception when it meets corrupted records.</li> | ||
| * </ul> | ||
| * | ||
| * @since 2.0.0 | ||
| */ | ||
| def csv(path: String): DataFrame = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| csv(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads a CSV file and returns the result as a [[DataFrame]]. | ||
| * | ||
|
|
@@ -381,6 +467,22 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| @scala.annotation.varargs | ||
| def csv(paths: String*): DataFrame = format("csv").load(paths : _*) | ||
|
|
||
| /** | ||
| * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty | ||
| * [[DataFrame]] if no paths are passed in. | ||
| * | ||
| * You can set the following Parquet-specific option(s) for reading Parquet files: | ||
| * <li>`mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets | ||
| * whether we should merge schemas collected from all Parquet part-files. This will override | ||
| * `spark.sql.parquet.mergeSchema`.</li> | ||
| * | ||
| * @since 2.0.0 | ||
| */ | ||
| def parquet(path: String): DataFrame = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| parquet(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty | ||
| * [[DataFrame]] if no paths are passed in. | ||
|
|
@@ -404,7 +506,20 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| * @since 1.5.0 | ||
| * @note Currently, this method can only be used after enabling Hive support. | ||
| */ | ||
| def orc(path: String): DataFrame = format("orc").load(path) | ||
| def orc(path: String): DataFrame = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| orc(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads an ORC file and returns the result as a [[DataFrame]]. | ||
| * | ||
| * @param paths input paths | ||
| * @since 2.0.0 | ||
| * @note Currently, this method can only be used after enabling Hive support. | ||
| */ | ||
| @scala.annotation.varargs | ||
| def orc(paths: String*): DataFrame = format("orc").load(paths: _*) | ||
|
|
||
| /** | ||
| * Returns the specified table as a [[DataFrame]]. | ||
|
|
@@ -430,12 +545,57 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { | |
| * spark.read().text("/path/to/spark/README.md") | ||
| * }}} | ||
| * | ||
| * @param paths input path | ||
| * @param path input path | ||
| * @since 2.0.0 | ||
| */ | ||
| def text(path: String): DataFrame = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| text(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named | ||
| * "value", and followed by partitioned columns if there are any. | ||
| * | ||
| * Each line in the text files is a new row in the resulting DataFrame. For example: | ||
| * {{{ | ||
| * // Scala: | ||
| * spark.read.text("/path/to/spark/README.md") | ||
| * | ||
| * // Java: | ||
| * spark.read().text("/path/to/spark/README.md") | ||
| * }}} | ||
| * | ||
| * @param paths input paths | ||
| * @since 1.6.0 | ||
| */ | ||
| @scala.annotation.varargs | ||
| def text(paths: String*): DataFrame = format("text").load(paths : _*) | ||
|
|
||
| /** | ||
| * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset | ||
| * contains a single string column named "value". | ||
| * | ||
| * If the directory structure of the text files contains partitioning information, those are | ||
| * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. | ||
| * | ||
| * Each line in the text files is a new element in the resulting Dataset. For example: | ||
| * {{{ | ||
| * // Scala: | ||
| * spark.read.textFile("/path/to/spark/README.md") | ||
| * | ||
| * // Java: | ||
| * spark.read().textFile("/path/to/spark/README.md") | ||
| * }}} | ||
| * | ||
| * @param path input path | ||
| * @since 2.0.0 | ||
| */ | ||
| def textFile(path: String): Dataset[String] = { | ||
| // This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
| textFile(Seq(path): _*) | ||
| } | ||
|
|
||
| /** | ||
| * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset | ||
| * contains a single string column named "value". | ||
|
|
||

There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
deduped.