-
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-50031][SQL] Add the TryParseUrl
expression
#48500
Changes from 4 commits
4656fbd
fbc201b
f49745a
5793bff
3122ac6
a2a1e8c
5071226
470f446
9f2ccc6
3513600
6656eaa
5434b30
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -583,6 +583,7 @@ URL Functions | |
:toctree: api/ | ||
|
||
parse_url | ||
try_parse_url | ||
url_decode | ||
url_encode | ||
try_url_decode | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -12912,6 +12912,108 @@ def substr( | |
return _invoke_function_over_columns("substr", str, pos) | ||
|
||
|
||
@_try_remote_functions | ||
def try_parse_url( | ||
url: "ColumnOrName", partToExtract: "ColumnOrName", key: Optional["ColumnOrName"] = None | ||
) -> Column: | ||
""" | ||
URL function: Extracts a specified part from a URL. If a key is provided, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you update this description to give the similar message as other try functions? |
||
it returns the associated query parameter value. | ||
|
||
.. versionadded:: 4.0.0 | ||
|
||
Parameters | ||
---------- | ||
url : :class:`~pyspark.sql.Column` or str | ||
A column of strings, each representing a URL. | ||
partToExtract : :class:`~pyspark.sql.Column` or str | ||
A column of strings, each representing the part to extract from the URL. | ||
key : :class:`~pyspark.sql.Column` or str, optional | ||
A column of strings, each representing the key of a query parameter in the URL. | ||
|
||
Returns | ||
------- | ||
:class:`~pyspark.sql.Column` | ||
A new column of strings, each representing the value of the extracted part from the URL. | ||
|
||
Examples | ||
-------- | ||
Example 1: Extracting the query part from a URL | ||
|
||
>>> from pyspark.sql import functions as sf | ||
>>> df = spark.createDataFrame( | ||
... [("https://spark.apache.org/path?query=1", "QUERY")], | ||
... ["url", "part"] | ||
... ) | ||
>>> df.select(sf.parse_url(df.url, df.part)).show() | ||
+--------------------+ | ||
|parse_url(url, part)| | ||
+--------------------+ | ||
| query=1| | ||
+--------------------+ | ||
|
||
Example 2: Extracting the value of a specific query parameter from a URL | ||
|
||
>>> from pyspark.sql import functions as sf | ||
>>> df = spark.createDataFrame( | ||
... [("https://spark.apache.org/path?query=1", "QUERY", "query")], | ||
... ["url", "part", "key"] | ||
... ) | ||
>>> df.select(sf.parse_url(df.url, df.part, df.key)).show() | ||
+-------------------------+ | ||
|parse_url(url, part, key)| | ||
+-------------------------+ | ||
| 1| | ||
+-------------------------+ | ||
|
||
Example 3: Extracting the protocol part from a URL | ||
|
||
>>> from pyspark.sql import functions as sf | ||
>>> df = spark.createDataFrame( | ||
... [("https://spark.apache.org/path?query=1", "PROTOCOL")], | ||
... ["url", "part"] | ||
... ) | ||
>>> df.select(sf.parse_url(df.url, df.part)).show() | ||
+--------------------+ | ||
|parse_url(url, part)| | ||
+--------------------+ | ||
| https| | ||
+--------------------+ | ||
|
||
Example 4: Extracting the host part from a URL | ||
|
||
>>> from pyspark.sql import functions as sf | ||
>>> df = spark.createDataFrame( | ||
... [("https://spark.apache.org/path?query=1", "HOST")], | ||
... ["url", "part"] | ||
... ) | ||
>>> df.select(sf.parse_url(df.url, df.part)).show() | ||
+--------------------+ | ||
|parse_url(url, part)| | ||
+--------------------+ | ||
| spark.apache.org| | ||
+--------------------+ | ||
|
||
Example 5: Extracting the path part from a URL | ||
|
||
>>> from pyspark.sql import functions as sf | ||
>>> df = spark.createDataFrame( | ||
... [("https://spark.apache.org/path?query=1", "PATH")], | ||
... ["url", "part"] | ||
... ) | ||
>>> df.select(sf.parse_url(df.url, df.part)).show() | ||
+--------------------+ | ||
|parse_url(url, part)| | ||
+--------------------+ | ||
| /path| | ||
+--------------------+ | ||
""" | ||
if key is not None: | ||
return _invoke_function_over_columns("try_parse_url", url, partToExtract, key) | ||
else: | ||
return _invoke_function_over_columns("try_parse_url", url, partToExtract) | ||
|
||
|
||
@_try_remote_functions | ||
def parse_url( | ||
url: "ColumnOrName", partToExtract: "ColumnOrName", key: Optional["ColumnOrName"] = None | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -169,6 +169,35 @@ object ParseUrl { | |
private val REGEXSUBFIX = "=([^&]*)" | ||
} | ||
|
||
/** | ||
* Extracts a part from a URL | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This comment is useless. Let's remove it. |
||
*/ | ||
@ExpressionDescription( | ||
usage = "_FUNC_(url, partToExtract[, key]) - Extracts a part from a URL.", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
examples = """ | ||
Examples: | ||
> SELECT _FUNC_('http://spark.apache.org/path?query=1', 'HOST'); | ||
spark.apache.org | ||
> SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY'); | ||
query=1 | ||
> SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query'); | ||
1 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you show an example of the NULL result?
MaxGekk marked this conversation as resolved.
Show resolved
Hide resolved
|
||
""", | ||
since = "4.0.0", | ||
group = "url_funcs") | ||
case class TryParseUrl(params: Seq[Expression], replacement: Expression) | ||
extends RuntimeReplaceable with InheritAnalysisRules { | ||
def this(children: Seq[Expression]) = this(children, ParseUrl(children, failOnError = false)) | ||
|
||
override def prettyName: String = "try_parse_url" | ||
|
||
override def parameters: Seq[Expression] = params | ||
|
||
override protected def withNewChildInternal(newChild: Expression): Expression = { | ||
copy(replacement = newChild) | ||
} | ||
} | ||
|
||
/** | ||
* Extracts a part from a URL | ||
*/ | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -84,6 +84,50 @@ class UrlFunctionsSuite extends QueryTest with SharedSparkSession { | |
} | ||
} | ||
|
||
test("url try_parse_url function") { | ||
|
||
def testUrl(url: String, expected: Row): Unit = { | ||
checkAnswer(Seq[String]((url)).toDF("url").selectExpr( | ||
"try_parse_url(url, 'HOST')", "try_parse_url(url, 'PATH')", | ||
"try_parse_url(url, 'QUERY')", "try_parse_url(url, 'REF')", | ||
"try_parse_url(url, 'PROTOCOL')", "try_parse_url(url, 'FILE')", | ||
"try_parse_url(url, 'AUTHORITY')", "try_parse_url(url, 'USERINFO')", | ||
"try_parse_url(url, 'QUERY', 'query')"), expected) | ||
} | ||
|
||
testUrl( | ||
"http://[email protected]/path?query=1#Ref", | ||
Row("spark.apache.org", "/path", "query=1", "Ref", | ||
"http", "/path?query=1", "[email protected]", "userinfo", "1")) | ||
|
||
testUrl( | ||
"https://use%20r:pas%[email protected]/dir%20/pa%20th.HTML?query=x%20y&q2=2#Ref%20two", | ||
Row("example.com", "/dir%20/pa%20th.HTML", "query=x%20y&q2=2", "Ref%20two", | ||
"https", "/dir%20/pa%20th.HTML?query=x%20y&q2=2", "use%20r:pas%[email protected]", | ||
"use%20r:pas%20s", "x%20y")) | ||
|
||
testUrl( | ||
"http://user:pass@host", | ||
Row("host", "", null, null, "http", "", "user:pass@host", "user:pass", null)) | ||
|
||
testUrl( | ||
"http://user:pass@host/", | ||
Row("host", "/", null, null, "http", "/", "user:pass@host", "user:pass", null)) | ||
|
||
testUrl( | ||
"http://user:pass@host/?#", | ||
Row("host", "/", "", "", "http", "/?", "user:pass@host", "user:pass", null)) | ||
|
||
testUrl( | ||
"http://user:pass@host/file;param?query;p2", | ||
Row("host", "/file;param", "query;p2", null, "http", "/file;param?query;p2", | ||
"user:pass@host", "user:pass", null)) | ||
|
||
testUrl( | ||
"inva lid://user:pass@host/file;param?query;p2", | ||
Row(null, null, null, null, null, null, null, null, null)) | ||
} | ||
|
||
test("url encode/decode function") { | ||
def testUrl(url: String, fn: String, expected: Row): Unit = { | ||
checkAnswer(Seq[String]((url)).toDF("url") | ||
|
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.
can we add a one unittest in, e.g., test_functions.py? then the tests will be resued in both Spark connect and classic.