-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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-44732][SQL] Built-in XML data source support #41832
Closed
Closed
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
bef243c
Built-in xml data source implementation
sandip-db 5db61e8
Some import and scala style fixes
sandip-db e5b6cfd
Add library dependencies
sandip-db d3cdbcb
Resource file path fixes and change AnyFunSuite to SharedSparkSession…
sandip-db 21766cc
Exclude XML test resource files from license check
sandip-db e4a3300
Change import from scala.jdk.Collections to scala.collection.JavaConv…
sandip-db ba62d47
Switch to default logger
sandip-db bccc6b1
Switch to using SharedSparkSession in XmlSuite and few other XML tests
sandip-db File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -145,3 +145,4 @@ empty.proto | |
.*\.proto.bin | ||
LimitedInputStream.java | ||
TimSort.java | ||
xml-resources/* |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
113 changes: 113 additions & 0 deletions
113
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/DefaultSource.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |
62 changes: 62 additions & 0 deletions
62
...ore/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataToCatalyst.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Would need to register this to SQL expression. Can be done in a followup.