-
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
Conversation
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.
Sorry for missing the dev vote thread. In this case, IP clearance is necessary for bringing databricks/spark-xml
to an upstream project of ASF. Steps we may follow:
- Verify license dependencies of databricks/spark-xml.
- Determine if SGA, CCLA/ICLA is necessary for IP attribution.
- Ensure any new PMC/Committer is updated to align with the project.
- Conduct a private@s.a.o mail vote.
- Update the Incubator IP clearance page, e.g. skywalking-rocketbot-ui, pulsar-manager.
- Notify the Incubator via mail.
- Contact infra and follow up.
License-wise there is no problem because I wrote them. It's Apache License 2. I filed CCLA/ICLA when I became a committer. Through SPIP, we have reached a lazy consensus including PMC votes. IP clearance is for an external project but this is really a plugin that has very small codebase. We haven't done that for Avro, CSV, cloudpickle for example in which the codebase is really small, and we reviewed them line by line. In addition, we're NOT just porting it as is but release a modified version per Sparks internal interface that will allows a lot of features such as parttioned table. TBH it's more work and code to modify them than just porting it. |
Thanks for the explanation @HyukjinKwon. I'm OK with it if we already have precedents like arvo and csv |
@@ -589,6 +589,11 @@ | |||
"<errors>" |
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!
sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XMLFileFormat.scala
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XMLFileFormat.scala
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XMLFileFormat.scala
Outdated
Show resolved
Hide resolved
...src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala
Outdated
Show resolved
Hide resolved
...src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala
Outdated
Show resolved
Hide resolved
...src/main/scala/org/apache/spark/sql/execution/datasources/xml/parsers/StaxXmlGenerator.scala
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/util/XmlFile.scala
Outdated
Show resolved
Hide resolved
- Add stub for xml expressions in spark connect - Add exception for xml expressions in sql/functions and pyspark - Corressponding test fixes
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
Outdated
Show resolved
Hide resolved
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
Outdated
Show resolved
Hide resolved
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala
Show resolved
Hide resolved
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Show resolved
Hide resolved
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Show resolved
Hide resolved
@@ -83,21 +86,21 @@ private[xml] object StaxXmlGenerator { | |||
|
|||
def writeElement(dt: DataType, v: Any, options: XmlOptions): Unit = (dt, v) match { | |||
case (_, null) | (NullType, _) => writer.writeCharacters(options.nullValue) | |||
case (StringType, v: UTF8String) => writer.writeCharacters(v.toString) | |||
case (StringType, v: String) => writer.writeCharacters(v) | |||
case (TimestampType, v: Timestamp) => |
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.
I think you can remove this, and case (DecimalType(), v: java.math.BigDecimal) => writer.writeCharacters(v.toString)
(See also JacksonGenerator
and which types are being handled).
BTW, we would also add the type supports for TimestampNTZType
, YearMonthIntervalType
and DayTimeIntervalType
. But they are orthogonal and can be done separately.
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.
Most StringType arrive here with value of type UTF8String.
Also, JacksonGenerator supports DecimalType and I was planning to add support for DecimalType in a followup. Is that not required?
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.
Looks pretty good
@yaooqinn @HyukjinKwon has addressed your concern. Can you please approve? |
* {@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 comment
The 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 comment
The reason will be displayed to describe this comment to others. Learn more.
yes. Changed:
{@code fieldA [[data1, data2]]}
to
{@code fieldA [[data1], [data2]]}
* @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 comment
The reason will be displayed to describe this comment to others. Learn more.
the schema parameter can be StructType
or Column
, the options parameter can be scala or java map, or omitted. This means we need 6 overloads of from_xml
.
Does it really worth it? I know we did the same thing for from_json
, but this is really convoluted.
How about something like
TextParsingFunction.newBuilder()
.withSchema(...) // It has multiple overloads
.withOptions(...) // It has multiple overloads
.xml() // returns a Column
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 comment
The reason will be displayed to describe this comment to others. Learn more.
I think we can remove (Scala-specific)
signature with Scala map for now. For the same reason, we don't have that scala specific version of to_csv
, etc.
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.
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 comment
The 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.
copy(timeZoneId = Option(timeZoneId)) | ||
} | ||
override def nullSafeEval(xml: Any): Any = xml match { | ||
case arr: GenericArrayData => |
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.
why do we match this case if the handling is exactly the same with case arr: ArrayData
?
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 are right. It shouldn't be there. Can I address this in a follow-up?
https://issues.apache.org/jira/browse/SPARK-44810
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.
sure
Let me get this in first because @sandip-db seems like working on another followup. Let's address them in a followup. |
The tests passed. Merged to master. |
xmlInputFactory.createFilteredReader(eventReader, filter) | ||
} | ||
|
||
// Jackson parsers can be ranked according to their performance: |
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.
Let's also update the docs
s"${DateFormatter.defaultPattern}'T'HH:mm:ss[.SSS][XXX]" | ||
}) | ||
|
||
// SPARK-39731: Enables the backward compatible parsing behavior. |
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.
Here too
* @since | ||
*/ | ||
// scalastyle:on line.size.limit | ||
def from_xml(e: Column, schema: StructType, options: Map[String, String]): Column = withExpr { |
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.
TODOs:
- Scala and Python implementation for Spark Connect
- Python and R implementation
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.
* | ||
* @since 4.0.0 | ||
*/ | ||
def xml(path: String): DataFrame = format("xml").load(path) |
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.
* | ||
* @since 4.0.0 | ||
*/ | ||
def xml(path: String): DataFrame = { |
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.
* | ||
* @since 4.0.0 | ||
*/ | ||
def xml(path: String): Unit = { |
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.
* @since 4.0.0 | ||
*/ | ||
// scalastyle:on line.size.limit | ||
def schema_of_xml(xml: Column, options: java.util.Map[String, String]): Column = { |
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.
shall we at least have an overload with scala options?
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.
Actually this is same as schema_of_json
. I suggested to only have Java map one only for now .. to avoid having too many overloaded versions.
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.
The FileFormat integration part LGTM. I assume the parsing code is the same as before.
### What changes were proposed in this pull request? This is the second PR related to the built-in XML data source implementation ([jira](https://issues.apache.org/jira/browse/SPARK-44751)). The previous [PR](apache#41832) ported the spark-xml package. This PR addresses the following: - Implement FileFormat interface - Address the review comments in the previous [XML PR](apache#41832) - Moved from_xml and schema_of_xml to sql/functions - Moved ".xml" to DataFrameReader/DataFrameWriter - Removed old APIs like XmlRelation, XmlReader, etc. - StaxXmlParser changes: - Use FailureSafeParser - Convert 'Row' usage to 'InternalRow' - Convert String to UTF8String - Handle MapData and ArrayData for MapType and ArrayType respectively - Use TimestampFormatter to parse timestamp - Use DateFormatter to parse date - StaxXmlGenerator changes: - Convert 'Row' usage to 'InternalRow' - Handle UTF8String for StringType - Handle MapData and ArrayData for MapType and ArrayType respectively - Use TimestampFormatter to format timestamp - Use DateFormatter to format date - Update XML tests accordingly because of the above changes ### Why are the changes needed? These changes are required to bring XML data source capability at par with CSV and JSON and supports features like streaming, which requires FileFormat interface to be implemented. ### Does this PR introduce _any_ user-facing change? Yes, it adds support for XML data source. ### How was this patch tested? - Ran all the XML unit tests. - Github Action Closes apache#42462 from sandip-db/xml-file-format-master. Authored-by: Sandip Agarwala <131817656+sandip-db@users.noreply.github.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
### What changes were proposed in this pull request? This is the second PR related to the built-in XML data source implementation ([jira](https://issues.apache.org/jira/browse/SPARK-44751)). The previous [PR](apache#41832) ported the spark-xml package. This PR addresses the following: - Implement FileFormat interface - Address the review comments in the previous [XML PR](apache#41832) - Moved from_xml and schema_of_xml to sql/functions - Moved ".xml" to DataFrameReader/DataFrameWriter - Removed old APIs like XmlRelation, XmlReader, etc. - StaxXmlParser changes: - Use FailureSafeParser - Convert 'Row' usage to 'InternalRow' - Convert String to UTF8String - Handle MapData and ArrayData for MapType and ArrayType respectively - Use TimestampFormatter to parse timestamp - Use DateFormatter to parse date - StaxXmlGenerator changes: - Convert 'Row' usage to 'InternalRow' - Handle UTF8String for StringType - Handle MapData and ArrayData for MapType and ArrayType respectively - Use TimestampFormatter to format timestamp - Use DateFormatter to format date - Update XML tests accordingly because of the above changes ### Why are the changes needed? These changes are required to bring XML data source capability at par with CSV and JSON and supports features like streaming, which requires FileFormat interface to be implemented. ### Does this PR introduce _any_ user-facing change? Yes, it adds support for XML data source. ### How was this patch tested? - Ran all the XML unit tests. - Github Action Closes apache#42462 from sandip-db/xml-file-format-master. Authored-by: Sandip Agarwala <131817656+sandip-db@users.noreply.github.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
What changes were proposed in this pull request?
This is the second PR related to the built-in XML data source implementation (jira).
The previous PR ported the spark-xml package.
This PR addresses the following:
Why are the changes needed?
These changes are required to bring XML data source capability at par with CSV and JSON and supports features like streaming, which requires FileFormat interface to be implemented.
Does this PR introduce any user-facing change?
Yes, it adds support for XML data source.
How was this patch tested?