Skip to content
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

Closed
wants to merge 12 commits into from

Conversation

sandip-db
Copy link
Contributor

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:

  • Implement FileFormat interface
  • Address the review comments in the previous XML PR
  • 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

Copy link
Member

@yaooqinn yaooqinn left a 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:

  1. Verify license dependencies of databricks/spark-xml.
  2. Determine if SGA, CCLA/ICLA is necessary for IP attribution.
  3. Ensure any new PMC/Committer is updated to align with the project.
  4. Conduct a private@s.a.o mail vote.
  5. Update the Incubator IP clearance page, e.g. skywalking-rocketbot-ui, pulsar-manager.
  6. Notify the Incubator via mail.
  7. Contact infra and follow up.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Aug 12, 2023

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.

@yaooqinn
Copy link
Member

Thanks for the explanation @HyukjinKwon. I'm OK with it if we already have precedents like arvo and csv

@@ -589,6 +589,11 @@
"<errors>"

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.

Copy link
Contributor Author

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.

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!

- Add stub for xml expressions in spark connect
- Add exception for xml expressions in sql/functions and pyspark
- Corressponding test fixes
@github-actions github-actions bot added the BUILD label Aug 17, 2023
@@ -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) =>
Copy link
Member

@HyukjinKwon HyukjinKwon Aug 21, 2023

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.

Copy link
Contributor Author

@sandip-db sandip-db Aug 21, 2023

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?

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks pretty good

@sandip-db
Copy link
Contributor Author

Thanks for the explanation @HyukjinKwon. I'm OK with it if we already have precedents like arvo and csv

@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>}
Copy link
Contributor

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?

Copy link
Contributor Author

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 =
Copy link
Contributor

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

Copy link
Member

@HyukjinKwon HyukjinKwon Aug 22, 2023

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.

Copy link
Contributor Author

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.

Copy link
Member

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 =>
Copy link
Contributor

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?

Copy link
Contributor Author

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

@HyukjinKwon
Copy link
Member

Let me get this in first because @sandip-db seems like working on another followup. Let's address them in a followup.

@HyukjinKwon
Copy link
Member

The tests passed.

Merged to master.

xmlInputFactory.createFilteredReader(eventReader, filter)
}

// Jackson parsers can be ranked according to their performance:
Copy link
Member

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.
Copy link
Member

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 {
Copy link
Member

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

Copy link
Contributor Author

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)
Copy link
Member

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 = {
Copy link
Member

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 = {
Copy link
Member

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 = {
Copy link
Contributor

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?

Copy link
Member

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.

Copy link
Contributor

@cloud-fan cloud-fan left a 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.

valentinp17 pushed a commit to valentinp17/spark that referenced this pull request Aug 24, 2023
### 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>
ragnarok56 pushed a commit to ragnarok56/spark that referenced this pull request Mar 2, 2024
### 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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants