sandip-db commented on code in PR #42462:
URL: https://github.com/apache/spark/pull/42462#discussion_r1296723330
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala:
##########
@@ -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]]}
Review Comment:
You are right. I checked the code and it generates only one
<fieldA>...</fieldA>
Its a typo carried over from
[here](https://github.com/databricks/spark-xml/commit/13539b41d6054442824457c67f0150e00ae353ea#diff-6571ce640b4e8e7d4b308679755e5f38503a0e2cec0e0ea4ee7d507afd181176R75)
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -7227,6 +7227,150 @@ 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 a `StructType` with 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 =
+ from_xml(e, lit(schema.toDDL), options.iterator)
+
+ // scalastyle:off line.size.limit
+
+ /**
+ * Parses a column containing a XML string into a `StructType` with 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: DataType, options: Map[String, String]):
Column =
+ from_xml(e, lit(schema.json), options.iterator)
+
+ // scalastyle:off line.size.limit
+
+ /**
+ * (Java-specific) Parses a column containing a XML string into a
`StructType` with 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 a `StructType` with 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])
+
+ /**
+ * Parses a column containing a XML string into a `StructType` with 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: DataType): 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)
+ }
+
Review Comment:
Not yet. I have filed a sub-task:
https://issues.apache.org/jira/browse/SPARK-44790
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]