Skip to content

Commit

Permalink
[SPARK-44732][SQL] Built-in XML data source support
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?
XML is a widely used data format. An external spark-xml package (https://github.com/databricks/spark-xml) is available to read and write XML data in spark. Making spark-xml built-in will provide a better user experience for Spark SQL and structured streaming. The proposal is to inline code from spark-xml package.

The PR has the following commits:
i) The first commit has the following:
- Copy of spark-xml src files.
- Update license
- Scala style and format fixes
- Change AnyFunSuite to SparkFunSuite

ii) Miscellaneous import and scala style fixes.
iii) Add library dependencies
iv) Resource file path fixes and change AnyFunSuite to SharedSparkSession or SQLTestUtils
v) Exclude XML test resource files from license check
vi) Change import from scala.jdk.Collections to scala.collection.JavaConverters

### Why are the changes needed?
Built-in support for XML data source would provide better user experience than having to import an external package.

### Does this PR introduce _any_ user-facing change?
Yes, Add built-in support for XML data source.

### How was this patch tested?
Tested the new unit-tests that came with the imported spark-xml package.
Also ran ./dev/run-test

Closes apache#41832 from sandip-db/spark-xml-master.

Authored-by: Sandip Agarwala <131817656+sandip-db@users.noreply.github.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
  • Loading branch information
sandip-db authored and ragnarok56 committed Mar 2, 2024
1 parent 81faec0 commit bbbf276
Show file tree
Hide file tree
Showing 100 changed files with 10,639 additions and 0 deletions.
1 change: 1 addition & 0 deletions dev/.rat-excludes
Original file line number Diff line number Diff line change
Expand Up @@ -145,3 +145,4 @@ empty.proto
.*\.proto.bin
LimitedInputStream.java
TimSort.java
xml-resources/*
2 changes: 2 additions & 0 deletions dev/deps/spark-deps-hadoop-3-hive-2.3
Original file line number Diff line number Diff line change
Expand Up @@ -245,9 +245,11 @@ super-csv/2.2.0//super-csv-2.2.0.jar
threeten-extra/1.7.1//threeten-extra-1.7.1.jar
tink/1.9.0//tink-1.9.0.jar
transaction-api/1.1//transaction-api-1.1.jar
txw2/3.0.2//txw2-3.0.2.jar
univocity-parsers/2.9.1//univocity-parsers-2.9.1.jar
wildfly-openssl/1.1.3.Final//wildfly-openssl-1.1.3.Final.jar
xbean-asm9-shaded/4.23//xbean-asm9-shaded-4.23.jar
xmlschema-core/2.3.0//xmlschema-core-2.3.0.jar
xz/1.9//xz-1.9.jar
zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar
zookeeper-jute/3.6.3//zookeeper-jute-3.6.3.jar
Expand Down
12 changes: 12 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,8 @@
<codehaus.jackson.version>1.9.13</codehaus.jackson.version>
<fasterxml.jackson.version>2.15.2</fasterxml.jackson.version>
<fasterxml.jackson.databind.version>2.15.2</fasterxml.jackson.databind.version>
<ws.xmlschema.version>2.3.0</ws.xmlschema.version>
<org.glassfish.jaxb.txw2.version>3.0.2</org.glassfish.jaxb.txw2.version>
<snappy.version>1.1.10.3</snappy.version>
<netlib.ludovic.dev.version>3.0.3</netlib.ludovic.dev.version>
<commons-codec.version>1.16.0</commons-codec.version>
Expand Down Expand Up @@ -986,6 +988,16 @@
<artifactId>jackson-module-jaxb-annotations</artifactId>
<version>${fasterxml.jackson.version}</version>
</dependency>
<dependency>
<groupId>org.apache.ws.xmlschema</groupId>
<artifactId>xmlschema-core</artifactId>
<version>${ws.xmlschema.version}</version>
</dependency>
<dependency>
<groupId>org.glassfish.jaxb</groupId>
<artifactId>txw2</artifactId>
<version>${org.glassfish.jaxb.txw2.version}</version>
</dependency>
<dependency>
<groupId>org.glassfish.jersey.core</groupId>
<artifactId>jersey-server</artifactId>
Expand Down
8 changes: 8 additions & 0 deletions sql/core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,14 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>org.apache.ws.xmlschema</groupId>
<artifactId>xmlschema-core</artifactId>
</dependency>
<dependency>
<groupId>org.glassfish.jaxb</groupId>
<artifactId>txw2</artifactId>
</dependency>
<dependency>
<groupId>org.apache.xbean</groupId>
<artifactId>xbean-asm9-shaded</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ org.apache.spark.sql.execution.datasources.noop.NoopDataSource
org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2
org.apache.spark.sql.execution.datasources.v2.text.TextDataSourceV2
org.apache.spark.sql.execution.datasources.xml.DefaultSource
org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
org.apache.spark.sql.execution.streaming.sources.RateStreamProvider
org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.
*/
package org.apache.spark.sql.execution.datasources.xml

import org.apache.hadoop.fs.Path

import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
import org.apache.spark.sql.execution.datasources.xml.util.XmlFile
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.StructType

/**
* Provides access to XML data from pure SQL statements (i.e. for users of the
* JDBC server).
*/
class DefaultSource
extends RelationProvider
with SchemaRelationProvider
with CreatableRelationProvider
with DataSourceRegister {

/**
* Short alias for spark-xml data source.
*/
override def shortName(): String = "xml"

private def checkPath(parameters: Map[String, String]): String = {
parameters.getOrElse("path",
throw new IllegalArgumentException("'path' must be specified for XML data."))
}

/**
* Creates a new relation for data store in XML given parameters.
* Parameters have to include 'path'.
*/
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation = {
createRelation(sqlContext, parameters, null)
}

/**
* Creates a new relation for data store in XML given parameters and user supported schema.
* Parameters have to include 'path'.
*/
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: StructType): XmlRelation = {
val path = checkPath(parameters)
// We need the `charset` and `rowTag` before creating the relation.
val (charset, rowTag) = {
val options = XmlOptions(parameters)
(options.charset, options.rowTag)
}

val paramsWithTZ =
sqlContext.sparkContext.getConf.getOption("spark.sql.session.timeZone") match {
case Some(tz) => parameters.updated("timezone", tz)
case None => parameters
}

XmlRelation(
() => XmlFile.withCharset(sqlContext.sparkContext, path, charset, rowTag),
Some(path),
paramsWithTZ,
schema)(sqlContext)
}

override def createRelation(
sqlContext: SQLContext,
mode: SaveMode,
parameters: Map[String, String],
data: DataFrame): BaseRelation = {
val path = checkPath(parameters)
val filesystemPath = new Path(path)
val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
val doSave = if (fs.exists(filesystemPath)) {
mode match {
case SaveMode.Append =>
throw new IllegalArgumentException(
s"Append mode is not supported by ${this.getClass.getCanonicalName}")
case SaveMode.Overwrite =>
fs.delete(filesystemPath, true)
true
case SaveMode.ErrorIfExists =>
throw new IllegalArgumentException(s"path $path already exists.")
case SaveMode.Ignore => false
}
} else {
true
}
if (doSave) {
// Only save data when the save mode is not ignore.
XmlFile.saveAsXmlFile(data, filesystemPath.toString, parameters)
}
createRelation(sqlContext, parameters, data.schema)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.
*/
package org.apache.spark.sql.execution.datasources.xml

import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.execution.datasources.xml.parsers.StaxXmlParser
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String

case class XmlDataToCatalyst(
child: Expression,
schema: DataType,
options: XmlOptions)
extends UnaryExpression with CodegenFallback with ExpectsInputTypes {

override lazy val dataType: DataType = schema

@transient
lazy val rowSchema: StructType = schema match {
case st: StructType => st
case ArrayType(st: StructType, _) => st
}

override def nullSafeEval(xml: Any): Any = xml match {
case string: UTF8String =>
CatalystTypeConverters.convertToCatalyst(
StaxXmlParser.parseColumn(string.toString, rowSchema, options))
case string: String =>
StaxXmlParser.parseColumn(string, rowSchema, options)
case arr: GenericArrayData =>
CatalystTypeConverters.convertToCatalyst(
arr.array.map(s => StaxXmlParser.parseColumn(s.toString, rowSchema, options)))
case arr: Array[_] =>
arr.map(s => StaxXmlParser.parseColumn(s.toString, rowSchema, options))
case _ => null
}

override def inputTypes: Seq[DataType] = schema match {
case _: StructType => Seq(StringType)
case ArrayType(_: StructType, _) => Seq(ArrayType(StringType))
}

// Overrides, in Spark 3.2.0+
protected def withNewChildInternal(newChild: Expression): XmlDataToCatalyst = copy(newChild)
}
Loading

0 comments on commit bbbf276

Please sign in to comment.