Skip to content

feat: implement parse_url#4350

Draft
parthchandra wants to merge 3 commits into
apache:mainfrom
parthchandra:parse_url
Draft

feat: implement parse_url#4350
parthchandra wants to merge 3 commits into
apache:mainfrom
parthchandra:parse_url

Conversation

@parthchandra
Copy link
Copy Markdown
Contributor

Which issue does this PR close?

Part of #4150.

Rationale for this change

parse_url is a commonly used Spark SQL function that currently falls back to the JVM. Adding native support lets Comet accelerate queries that use it. The native implementation in DataFusion already exists (datafusion-spark crate); this PR wires the Spark expression through the Comet serde layer to the native
UDFs.

Because the native implementation has known divergences from Spark on certain edge cases (percent-encoding in QUERY values, FILE part for path-less URLs — tracked in apache/datafusion#21943), the expression is marked Incompatible and gated behind
spark.comet.expression.ParseUrl.allowIncompatible=true.

Note: #4152 implemented parse_url but was closed in favor of #4231 which did not implement parse_url

What changes are included in this PR?

  • Serde handler (url.scala): New CometParseUrl handler that maps ParseUrl to either parse_url or try_parse_url native UDF depending on failOnError (ANSI mode).
  • Spark 4.x shim (both spark-4.0 and spark-4.1 CometExprShim.scala): Handle Invoke(Literal(ParseUrlEvaluator), "evaluate", args) — the rewritten form of ParseUrl in Spark 4.x — by reconstructing a ParseUrl node and re-dispatching through the serde framework. Propagates EXTENSION_INFO tags so fallback reasons
    are reported correctly.
  • UDF registration (jni_api.rs): Register SparkParseUrl and SparkTryParseUrl UDFs with the DataFusion session context.
  • Expression map (QueryPlanSerde.scala): Add urlExpressions map and include it in the combined expression lookup.
  • Doc generation (GenerateDocs.scala): Add "url" category so compatibility docs are auto-generated.
  • Compatibility docs (url.md): Template for auto-generated URL expression compatibility table.

How are these changes tested?

Three new SQL test files covering different configurations:

  • parse_url.sql — Default config (allowIncompatible=false): verifies the expression falls back with the expected incompatibility reason.
  • parse_url_enabled.sql — With allowIncompatible=true: exercises all URL parts (HOST, PATH, QUERY, PROTOCOL, REF, AUTHORITY, USERINFO, FILE), literal and column-valued arguments, NULL handling, malformed URLs, column-valued part keys, and known edge cases (two tests marked ignore for documented divergences).
  • parse_url_ansi.sql — ANSI mode (failOnError=true): verifies the parse_url (non-try) native path works for valid URLs.

All tests pass on both Spark 4.0 and Spark 4.1 profiles.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant