From a484826244afcc3b5aa0e9e6155088a8d7c9e974 Mon Sep 17 00:00:00 2001 From: tangjiafu Date: Tue, 24 Oct 2023 17:34:41 +0900 Subject: [PATCH] [SPARK-44752][SQL] XML: Update Spark Docs ### What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-44752 ### Why are the changes needed? The XML data source is basically supported, but the XML example and document page are not yet available ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Annotated the methods of other data sources, click on 'run' in the idea to run ### Was this patch authored or co-authored using generative AI tooling? It was written by my Rubik's Cube JSON and CSV Closes #43350 from laglangyue/xml_example_doc. Lead-authored-by: tangjiafu Co-authored-by: laglangyue Signed-off-by: Hyukjin Kwon --- dev/.rat-excludes | 1 + docs/_data/menu-sql.yaml | 2 + docs/sql-data-sources-xml.md | 232 ++++++++++++++++++ docs/sql-data-sources.md | 1 + .../sql/JavaSQLDataSourceExample.java | 53 ++++ examples/src/main/python/sql/datasource.py | 49 ++++ examples/src/main/resources/people.xml | 15 ++ .../examples/sql/SQLDataSourceExample.scala | 54 +++- 8 files changed, 405 insertions(+), 2 deletions(-) create mode 100644 docs/sql-data-sources-xml.md create mode 100644 examples/src/main/resources/people.xml diff --git a/dev/.rat-excludes b/dev/.rat-excludes index d755c0ff749c4..a27319f16aaae 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -143,3 +143,4 @@ LimitedInputStream.java TimSort.java xml-resources/* loose_version.py +people.xml diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index ff93f09a83ce1..4125860642294 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -36,6 +36,8 @@ url: sql-data-sources-csv.html - text: Text Files url: sql-data-sources-text.html + - text: XML Files + url: sql-data-sources-xml.html - text: Hive Tables url: sql-data-sources-hive-tables.html - text: JDBC To Other Databases diff --git a/docs/sql-data-sources-xml.md b/docs/sql-data-sources-xml.md new file mode 100644 index 0000000000000..4537ade43d2cf --- /dev/null +++ b/docs/sql-data-sources-xml.md @@ -0,0 +1,232 @@ +--- +layout: global +title: XML Files +displayTitle: XML Files +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +Spark SQL provides `spark.read().xml("file_1_path","file_2_path")` to read a file or directory of files in XML format into a Spark DataFrame, and `dataframe.write().xml("path")` to write to a xml file. When reading a XML file, the `rowTag` option must be specified to indicate the XML element that maps to a `DataFrame row`. The option() function can be used to customize the behavior of reading or writing, such as controlling behavior of the XML attributes, XSD validation, compression, and so on. + +
+ +
+{% include_example xml_dataset python/sql/datasource.py %} +
+ +
+{% include_example xml_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
+ +
+{% include_example xml_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
+ +
+ +## Data Source Option + +Data source options of XML can be set via: + +* the `.option`/`.options` methods of + * `DataFrameReader` + * `DataFrameWriter` + * `DataStreamReader` + * `DataStreamWriter` +* the built-in functions below + * `from_xml` + * `to_xml` + * `schema_of_xml` +* `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaningScope
rowTagThe row tag of your xml files to treat as a row. For example, in this xml: + <books><book></book>...</books> + the appropriate value would be book. This is a required option for both read and write. + read
samplingRatio1.0Defines fraction of rows used for schema inferring. XML built-in functions ignore this option.read
excludeAttributefalseWhether to exclude attributes in elements.read
modePERMISSIVEAllows a mode for dealing with corrupt records during parsing.
+
    +
  • PERMISSIVE: when it meets a corrupted record, puts the malformed string into a field configured by columnNameOfCorruptRecord, and sets malformed fields to null. To keep corrupt records, an user can set a string type field named columnNameOfCorruptRecord in an user-defined schema. If a schema does not have the field, it drops corrupt records during parsing. When inferring a schema, it implicitly adds a columnNameOfCorruptRecord field in an output schema.
  • +
  • DROPMALFORMED: ignores the whole corrupted records. This mode is unsupported in the JSON built-in functions.
  • +
  • FAILFAST: throws an exception when it meets corrupted records.
  • +
+
read
inferSchematrueIf true, attempts to infer an appropriate type for each resulting DataFrame column. If false, all resulting columns are of string type. Default is true. XML built-in functions ignore this option.read
columnNameOfCorruptRecordspark.sql.columnNameOfCorruptRecordAllows renaming the new field having a malformed string created by PERMISSIVE mode.read
attributePrefix_The prefix for attributes to differentiate attributes from elements. This will be the prefix for field names. Default is _. Can be empty for reading XML, but not for writing.read/write
valueTag_VALUEThe tag used for the value when there are attributes in the element having no child.read/write
encodingUTF-8For reading, decodes the XML files by the given encoding type. For writing, specifies encoding (charset) of saved XML files. XML built-in functions ignore this option. read/write
ignoreSurroundingSpacesfalseDefines whether surrounding whitespaces from values being read should be skipped.read
rowValidationXSDPathnullPath to an optional XSD file that is used to validate the XML for each row individually. Rows that fail to validate are treated like parse errors as above. The XSD does not otherwise affect the schema provided, or inferred.read
ignoreNamespacefalseIf true, namespaces prefixes on XML elements and attributes are ignored. Tags <abc:author> and <def:author> would, for example, be treated as if both are just <author>. Note that, at the moment, namespaces cannot be ignored on the rowTag element, only its children. Note that XML parsing is in general not namespace-aware even if false.read
timeZone(value of spark.sql.session.timeZone configuration)Sets the string that indicates a time zone ID to be used to format timestamps in the XML datasources or partition values. The following formats of timeZone are supported:
+
    +
  • Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
  • +
  • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00', also 'UTC' and 'Z' are supported as aliases of '+00:00'.
  • +
+ Other short names like 'CST' are not recommended to use because they can be ambiguous. +
read/write
timestampFormatyyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]Sets the string that indicates a timestamp format. Custom date formats follow the formats at datetime pattern. This applies to timestamp type.read/write
dateFormatyyyy-MM-ddSets the string that indicates a date format. Custom date formats follow the formats at datetime pattern. This applies to date type.read/write
localeen-USSets a locale as a language tag in IETF BCP 47 format. For instance, locale is used while parsing dates and timestamps. read/write
rootTagROWSRoot tag of the xml files. For example, in this xml: + <books><book></book>...</books> + the appropriate value would be books. It can include basic attributes by specifying a value like 'books' + write
declarationversion="1.0" + encoding="UTF-8" + standalone="yes"Content of XML declaration to write at the start of every output XML file, before the rootTag. For example, a value of foo causes to be written. Set to empty string to suppresswrite
arrayElementNameitemName of XML element that encloses each element of an array-valued column when writing.write
nullValuenullSets the string representation of a null value. Default is string null. When this is null, it does not write attributes and elements for fields.read/write
wildcardColNamexs_anyName of a column existing in the provided schema which is interpreted as a 'wildcard'. It must have type string or array of strings. It will match any XML child element that is not otherwise matched by the schema. The XML of the child becomes the string value of the column. If an array, then all unmatched elements will be returned as an array of strings. As its name implies, it is meant to emulate XSD's xs:any type.read
compressionnoneCompression codec to use when saving to file. This can be one of the known case-insensitive shortened names (none, bzip2, gzip, lz4, snappy and deflate). XML built-in functions ignore this option.write
+Other generic options can be found in Generic File Source Options. diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md index 041919b071d4d..a2bedd178abb0 100644 --- a/docs/sql-data-sources.md +++ b/docs/sql-data-sources.md @@ -48,6 +48,7 @@ goes into specific options that are available for the built-in data sources. * [JSON Files](sql-data-sources-json.html) * [CSV Files](sql-data-sources-csv.html) * [Text Files](sql-data-sources-text.html) +* [XML Files](sql-data-sources-xml.html) * [Hive Tables](sql-data-sources-hive-tables.html) * [Specifying storage format for Hive tables](sql-data-sources-hive-tables.html#specifying-storage-format-for-hive-tables) * [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore) diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index c0960540b4953..efbed9e591eca 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -17,9 +17,11 @@ package org.apache.spark.examples.sql; // $example on:schema_merging$ +import com.google.common.collect.Lists; import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; // $example off:schema_merging$ import java.util.Properties; @@ -109,6 +111,7 @@ public static void main(String[] args) { runCsvDatasetExample(spark); runTextDatasetExample(spark); runJdbcDatasetExample(spark); + runXmlDatasetExample(spark); spark.stop(); } @@ -496,4 +499,54 @@ private static void runJdbcDatasetExample(SparkSession spark) { .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties); // $example off:jdbc_dataset$ } + + private static void runXmlDatasetExample(SparkSession spark) { + // $example on:xml_dataset$ + // Primitive types (Int, String, etc) and Product types (case classes) encoders are + // supported by importing this when creating a Dataset. + + // An XML dataset is pointed to by path. + // The path can be either a single xml file or more xml files + String path = "examples/src/main/resources/people.xml"; + Dataset peopleDF = spark.read().option("rowTag", "person").xml(path); + + // The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema(); + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people"); + + // SQL statements can be run by using the sql methods provided by spark + Dataset teenagerNamesDF = spark.sql( + "SELECT name FROM people WHERE age BETWEEN 13 AND 19"); + teenagerNamesDF.show(); + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + + // Alternatively, a DataFrame can be created for an XML dataset represented by a Dataset[String] + List xmlData = Collections.singletonList( + "" + + "laglangyueDeveloper28" + + ""); + Dataset otherPeopleDataset = spark.createDataset(Lists.newArrayList(xmlData), + Encoders.STRING()); + + Dataset otherPeople = spark.read() + .option("rowTag", "person") + .xml(otherPeopleDataset); + otherPeople.show(); + // +---+---------+----------+ + // |age| job| name| + // +---+---------+----------+ + // | 28|Developer|laglangyue| + // +---+---------+----------+ + // $example off:xml_dataset$ + + } } diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index c7522cb9d34c6..7a5dd9463c687 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -418,6 +418,54 @@ def jdbc_dataset_example(spark: SparkSession) -> None: # $example off:jdbc_dataset$ +def xml_dataset_example(spark: SparkSession) -> None: + # $example on:xml_dataset$ + # Primitive types (Int, String, etc) and Product types (case classes) encoders are + # supported by importing this when creating a Dataset. + # An XML dataset is pointed to by path. + # The path can be either a single xml file or more xml files + path = "examples/src/main/resources/people.xml" + peopleDF = spark.read.option("rowTag", "person").format("xml").load(path) + + # The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + # root + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) + + # Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + # SQL statements can be run by using the sql methods provided by spark + teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + # +------+ + # | name| + # +------+ + # |Justin| + # +------+ + + # Alternatively, a DataFrame can be created for an XML dataset represented by a Dataset[String] + xmlStrings = [""" + + laglangyue + Developer + 28 + + """] + xmlRDD = spark.sparkContext.parallelize(xmlStrings) + otherPeople = spark.read \ + .option("rowTag", "person") \ + .xml(xmlRDD) + otherPeople.show() + # +---+---------+----------+ + # |age| job| name| + # +---+---------+----------+ + # | 28|Developer|laglangyue| + # +---+---------+----------+ + # $example off:xml_dataset$ + + if __name__ == "__main__": spark = SparkSession \ .builder \ @@ -432,5 +480,6 @@ def jdbc_dataset_example(spark: SparkSession) -> None: csv_dataset_example(spark) text_dataset_example(spark) jdbc_dataset_example(spark) + xml_dataset_example(spark) spark.stop() diff --git a/examples/src/main/resources/people.xml b/examples/src/main/resources/people.xml new file mode 100644 index 0000000000000..11290e9747546 --- /dev/null +++ b/examples/src/main/resources/people.xml @@ -0,0 +1,15 @@ + + + + Michael + 29 + + + Andy + 30 + + + Justin + 19 + + diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 9b04994199df0..33121586fe101 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -39,6 +39,7 @@ object SQLDataSourceExample { runCsvDatasetExample(spark) runTextDatasetExample(spark) runJdbcDatasetExample(spark) + runXmlDatasetExample(spark) spark.stop() } @@ -108,7 +109,7 @@ object SQLDataSourceExample { // |file1.parquet| // +-------------+ val afterFilterDF = spark.read.format("parquet") - // Files modified after 06/01/2020 at 05:30 are allowed + // Files modified after 06/01/2020 at 05:30 are allowed .option("modifiedAfter", "2020-06-01T05:30:00") .load("examples/src/main/resources/dir1"); afterFilterDF.show(); @@ -308,7 +309,7 @@ object SQLDataSourceExample { // +-----+---+---------+ // You can also use options() to use multiple options - val df4 = spark.read.options(Map("delimiter"->";", "header"->"true")).csv(path) + val df4 = spark.read.options(Map("delimiter" -> ";", "header" -> "true")).csv(path) // "output" is a folder which contains multiple csv files and a _SUCCESS file. df3.write.csv("output") @@ -418,4 +419,53 @@ object SQLDataSourceExample { .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) // $example off:jdbc_dataset$ } + + private def runXmlDatasetExample(spark: SparkSession): Unit = { + // $example on:xml_dataset$ + // Primitive types (Int, String, etc) and Product types (case classes) encoders are + // supported by importing this when creating a Dataset. + import spark.implicits._ + // An XML dataset is pointed to by path. + // The path can be either a single xml file or more xml files + val path = "examples/src/main/resources/people.xml" + val peopleDF = spark.read.option("rowTag", "person").xml(path) + + // The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + // SQL statements can be run by using the sql methods provided by spark + val teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + + // Alternatively, a DataFrame can be created for a XML dataset represented by a Dataset[String] + val otherPeopleDataset = spark.createDataset( + """ + | + | laglangyue + | Developer + | 28 + | + |""".stripMargin :: Nil) + val otherPeople = spark.read + .option("rowTag", "person") + .xml(otherPeopleDataset) + otherPeople.show() + // +---+---------+----------+ + // |age| job| name| + // +---+---------+----------+ + // | 28|Developer|laglangyue| + // +---+---------+----------+ + // $example off:xml_dataset$ + } }