-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-44751][SQL] XML FileFormat Interface implementation #42462
Changes from all commits
1566b3e
44191f6
8803114
6ef36ef
544e20e
3dc2690
7d4bfb3
ea641f8
4cca597
6c201c9
0661283
54fafc8
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 |
---|---|---|
|
@@ -392,6 +392,46 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging | |
def csv(csvDataset: Dataset[String]): DataFrame = | ||
parse(csvDataset, ParseFormat.PARSE_FORMAT_CSV) | ||
|
||
/** | ||
* Loads a XML file and returns the result as a `DataFrame`. See the documentation on the other | ||
* overloaded `xml()` method for more details. | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
def xml(path: String): DataFrame = { | ||
sandip-db marked this conversation as resolved.
Show resolved
Hide resolved
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. |
||
// This method ensures that calls that explicit need single argument works, see SPARK-16009 | ||
xml(Seq(path): _*) | ||
} | ||
|
||
/** | ||
* Loads XML files 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 find the XML-specific options for reading XML files in <a | ||
* href="https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> | ||
sandip-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
* Data Source Option</a> in the version you use. | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
@scala.annotation.varargs | ||
def xml(paths: String*): DataFrame = format("xml").load(paths: _*) | ||
|
||
/** | ||
* Loads an `Dataset[String]` storing XML object and returns the result as a `DataFrame`. | ||
* | ||
* If the schema is not specified using `schema` function and `inferSchema` option is enabled, | ||
* this function goes through the input once to determine the input schema. | ||
* | ||
* @param xmlDataset | ||
* input Dataset with one XML object per record | ||
* @since 4.0.0 | ||
*/ | ||
def xml(xmlDataset: Dataset[String]): DataFrame = | ||
parse(xmlDataset, ParseFormat.PARSE_FORMAT_UNSPECIFIED) | ||
|
||
/** | ||
* Loads a Parquet file, returning the result as a `DataFrame`. See the documentation on the | ||
* other overloaded `parquet()` method for more details. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -470,6 +470,34 @@ final class DataFrameWriter[T] private[sql] (ds: Dataset[T]) { | |
format("csv").save(path) | ||
} | ||
|
||
/** | ||
* Saves the content of the `DataFrame` in XML format at the specified path. This is equivalent | ||
* to: | ||
* {{{ | ||
* format("xml").save(path) | ||
* }}} | ||
* | ||
* Note that writing a XML file from `DataFrame` having a field `ArrayType` with its element as | ||
* `ArrayType` would have an additional nested field for the element. For example, the | ||
* `DataFrame` having a field below, | ||
* | ||
* {@code fieldA [[data1], [data2]]} | ||
* | ||
* would produce a XML file below. { @code <fieldA> <item>data1</item> </fieldA> <fieldA> | ||
* <item>data2</item> </fieldA>} | ||
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. is this the fixed version? 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. yes. Changed: |
||
* | ||
* Namely, roundtrip in writing and reading can end up in different schema structure. | ||
* | ||
* You can find the XML-specific options for writing XML files in <a | ||
* href="https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> | ||
* Data Source Option</a> in the version you use. | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
def xml(path: String): Unit = { | ||
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. |
||
format("xml").save(path) | ||
} | ||
|
||
/////////////////////////////////////////////////////////////////////////////////////// | ||
// Builder pattern config options | ||
/////////////////////////////////////////////////////////////////////////////////////// | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -7227,6 +7227,112 @@ object functions { | |
*/ | ||
def to_csv(e: Column): Column = to_csv(e, Collections.emptyMap()) | ||
|
||
// scalastyle:off line.size.limit | ||
/** | ||
* Parses a column containing a XML string into the data type corresponding to the specified | ||
* schema. Returns `null`, in the case of an unparseable string. | ||
* | ||
* @param e | ||
* a string column containing XML data. | ||
* @param schema | ||
* the schema to use when parsing the XML string | ||
* @param options | ||
* options to control how the XML is parsed. accepts the same options and the XML data source. | ||
* See <a href= | ||
* "https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> Data | ||
* Source Option</a> in the version you use. | ||
* @group collection_funcs | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
// scalastyle:on line.size.limit | ||
def from_xml(e: Column, schema: StructType, options: Map[String, String]): Column = | ||
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. the schema parameter can be Does it really worth it? I know we did the same thing for How about something like
Anyway, it's unrelated to this PR. We can do it later. cc @HyukjinKwon 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 think we can remove 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. from_csv has just two. I can trim from_xml overloads too. Let me know. 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. Let's remove this signature with Scala map for now in a followup. |
||
from_xml(e, lit(schema.toDDL), options.iterator) | ||
|
||
// scalastyle:off line.size.limit | ||
|
||
/** | ||
* (Java-specific) Parses a column containing a XML string into the data type corresponding to | ||
* the specified schema. Returns `null`, in the case of an unparseable string. | ||
* | ||
* @param e | ||
* a string column containing XML data. | ||
* @param schema | ||
* the schema to use when parsing the XML string | ||
* @param options | ||
* options to control how the XML is parsed. accepts the same options and the XML data source. | ||
* See <a href= | ||
* "https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> Data | ||
* Source Option</a> in the version you use. | ||
* @group collection_funcs | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
// scalastyle:on line.size.limit | ||
def from_xml(e: Column, schema: Column, options: java.util.Map[String, String]): Column = | ||
from_xml(e, schema, options.asScala.iterator) | ||
|
||
/** | ||
* Parses a column containing a XML string into the data type corresponding to the specified | ||
* schema. Returns `null`, in the case of an unparseable string. | ||
* | ||
* @param e | ||
* a string column containing XML data. | ||
* @param schema | ||
* the schema to use when parsing the XML string | ||
* @group collection_funcs | ||
* | ||
* @since 4.0.0 | ||
*/ | ||
def from_xml(e: Column, schema: StructType): Column = | ||
from_xml(e, schema, Map.empty[String, String]) | ||
|
||
private def from_xml(e: Column, schema: Column, options: Iterator[(String, String)]): Column = { | ||
fnWithOptions("from_xml", options, e, schema) | ||
} | ||
|
||
/** | ||
* Parses a XML string and infers its schema in DDL format. | ||
* | ||
* @param xml | ||
* a XML string. | ||
* @group collection_funcs | ||
* @since 4.0.0 | ||
*/ | ||
def schema_of_xml(xml: String): Column = schema_of_xml(lit(xml)) | ||
|
||
/** | ||
* Parses a XML string and infers its schema in DDL format. | ||
* | ||
* @param xml | ||
* a foldable string column containing a XML string. | ||
* @group collection_funcs | ||
* @since 4.0.0 | ||
*/ | ||
def schema_of_xml(xml: Column): Column = Column.fn("schema_of_xml", xml) | ||
|
||
// scalastyle:off line.size.limit | ||
|
||
/** | ||
* Parses a XML string and infers its schema in DDL format using options. | ||
* | ||
* @param xml | ||
* a foldable string column containing XML data. | ||
* @param options | ||
* options to control how the xml is parsed. accepts the same options and the XML data source. | ||
* See <a href= | ||
* "https://spark.apache.org/docs/latest/sql-data-sources-xml.html#data-source-option"> Data | ||
* Source Option</a> in the version you use. | ||
* @return | ||
* a column with string literal containing schema in DDL format. | ||
* @group collection_funcs | ||
* @since 4.0.0 | ||
*/ | ||
// scalastyle:on line.size.limit | ||
def schema_of_xml(xml: Column, options: java.util.Map[String, String]): Column = { | ||
fnWithOptions("schema_of_xml", options.asScala.iterator, xml) | ||
} | ||
|
||
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. there is no 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. Not yet. I have filed a sub-task: https://issues.apache.org/jira/browse/SPARK-44790 |
||
/** | ||
* Returns the total number of elements in the array. The function returns null for null input. | ||
* | ||
|
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.
This is great! I was thinking upgrading the xml reader with data source v2 before but really stopped by the refactoring work involved. Thanks for adding it into the spark mainline to unify the interfaces and catch up with the main changes.
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.
Thanks for reviewing this PR.
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.
You're welcome! This is great!