Skip to content

Commit 00f1057

Browse files
sarutakdongjoon-hyun
authored andcommitted
[SPARK-50582][SQL][PYTHON] Add quote builtin function
### What changes were proposed in this pull request? This PR proposes to add `quote` builtin function, which encloses the given string by single quotes and prepends a backslash to each instance of single quote in the string. The implementation and behavior are same as Hive's ones. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=27362046#LanguageManualUDF-StringFunctions https://github.com/apache/hive/blob/3af4517eb8cfd9407ad34ed78a0b48b57dfaa264/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFQuote.java#L54 ### Why are the changes needed? For interoperability with Hive. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49191 from sarutak/quote-builtin-function. Authored-by: Kousuke Saruta <sarutak@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
1 parent 16784fa commit 00f1057

File tree

12 files changed

+195
-0
lines changed

12 files changed

+195
-0
lines changed

python/pyspark/sql/connect/functions/builtin.py

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3106,6 +3106,13 @@ def collation(col: "ColumnOrName") -> Column:
31063106
collation.__doc__ = pysparkfuncs.collation.__doc__
31073107

31083108

3109+
def quote(col: "ColumnOrName") -> Column:
3110+
return _invoke_function_over_columns("quote", col)
3111+
3112+
3113+
quote.__doc__ = pysparkfuncs.quote.__doc__
3114+
3115+
31093116
# Date/Timestamp functions
31103117

31113118

python/pyspark/sql/functions/builtin.py

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17656,6 +17656,37 @@ def collation(col: "ColumnOrName") -> Column:
1765617656
return _invoke_function_over_columns("collation", col)
1765717657

1765817658

17659+
@_try_remote_functions
17660+
def quote(col: "ColumnOrName") -> Column:
17661+
r"""Returns `str` enclosed by single quotes and each instance of
17662+
single quote in it is preceded by a backslash.
17663+
17664+
.. versionadded:: 4.1.0
17665+
17666+
Parameters
17667+
----------
17668+
col : :class:`~pyspark.sql.Column` or column name
17669+
target column to be quoted.
17670+
17671+
Returns
17672+
-------
17673+
:class:`~pyspark.sql.Column`
17674+
quoted string
17675+
17676+
Examples
17677+
--------
17678+
>>> from pyspark.sql import functions as sf
17679+
>>> df = spark.createDataFrame(["Don't"], "STRING")
17680+
>>> df.select("*", sf.quote("value")).show()
17681+
+-----+------------+
17682+
|value|quote(value)|
17683+
+-----+------------+
17684+
|Don't| 'Don\'t'|
17685+
+-----+------------+
17686+
"""
17687+
return _invoke_function_over_columns("quote", col)
17688+
17689+
1765917690
# ---------------------- Collection functions ------------------------------
1766017691

1766117692

sql/api/src/main/scala/org/apache/spark/sql/functions.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5082,6 +5082,15 @@ object functions {
50825082
*/
50835083
def right(str: Column, len: Column): Column = Column.fn("right", str, len)
50845084

5085+
/**
5086+
* Returns `str` enclosed by single quotes and each instance of single quote in it is preceded
5087+
* by a backslash.
5088+
*
5089+
* @group string_funcs
5090+
* @since 4.1.0
5091+
*/
5092+
def quote(str: Column): Column = Column.fn("quote", str)
5093+
50855094
//////////////////////////////////////////////////////////////////////////////////////////////
50865095
// DateTime functions
50875096
//////////////////////////////////////////////////////////////////////////////////////////////

sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -334,4 +334,12 @@ public static UTF8String randStr(XORShiftRandom rng, int length) {
334334
}
335335
return UTF8String.fromBytes(bytes);
336336
}
337+
338+
public static UTF8String quote(UTF8String str) {
339+
final String qtChar = "'";
340+
final String qtCharRep = "\\\\'";
341+
342+
String sp = str.toString().replaceAll(qtChar, qtCharRep);
343+
return UTF8String.fromString(qtChar + sp + qtChar);
344+
}
337345
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -613,6 +613,7 @@ object FunctionRegistry {
613613
expression[MakeValidUTF8]("make_valid_utf8"),
614614
expression[ValidateUTF8]("validate_utf8"),
615615
expression[TryValidateUTF8]("try_validate_utf8"),
616+
expression[Quote]("quote"),
616617

617618
// url functions
618619
expression[UrlEncode]("url_encode"),

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3719,3 +3719,46 @@ case class Luhncheck(input: Expression) extends RuntimeReplaceable with Implicit
37193719
override protected def withNewChildrenInternal(
37203720
newChildren: IndexedSeq[Expression]): Expression = copy(newChildren(0))
37213721
}
3722+
3723+
/**
3724+
* A function that prepends a backslash to each instance of single quote
3725+
* in the given string and encloses the result by single quotes.
3726+
*/
3727+
// scalastyle:off line.size.limit
3728+
@ExpressionDescription(
3729+
usage = "_FUNC_(str) - Returns `str` enclosed by single quotes and each instance of single quote in it is preceded by a backslash.",
3730+
examples = """
3731+
Examples:
3732+
> SELECT _FUNC_('Don\'t');
3733+
'Don\'t'
3734+
""",
3735+
since = "4.1.0",
3736+
group = "string_funcs")
3737+
// scalastyle:on line.size.limit
3738+
case class Quote(input: Expression)
3739+
extends UnaryExpression
3740+
with RuntimeReplaceable
3741+
with ImplicitCastInputTypes
3742+
with DefaultStringProducingExpression {
3743+
3744+
override lazy val replacement: Expression = StaticInvoke(
3745+
classOf[ExpressionImplUtils],
3746+
dataType,
3747+
"quote",
3748+
Seq(input),
3749+
inputTypes)
3750+
3751+
override def inputTypes: Seq[AbstractDataType] = {
3752+
Seq(StringTypeWithCollation(supportsTrimCollation = true))
3753+
}
3754+
3755+
override def nodeName: String = "quote"
3756+
3757+
override def nullable: Boolean = true
3758+
3759+
override def child: Expression = input
3760+
3761+
override protected def withNewChildInternal(newChild: Expression): Quote = {
3762+
copy(input = newChild)
3763+
}
3764+
}

sql/core/src/test/resources/sql-functions/sql-expression-schema.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -258,6 +258,7 @@
258258
| org.apache.spark.sql.catalyst.expressions.Pow | pow | SELECT pow(2, 3) | struct<pow(2, 3):double> |
259259
| org.apache.spark.sql.catalyst.expressions.Pow | power | SELECT power(2, 3) | struct<POWER(2, 3):double> |
260260
| org.apache.spark.sql.catalyst.expressions.Quarter | quarter | SELECT quarter('2016-08-31') | struct<quarter(2016-08-31):int> |
261+
| org.apache.spark.sql.catalyst.expressions.Quote | quote | SELECT quote('Don\'t') | struct<quote(Don't):string> |
261262
| org.apache.spark.sql.catalyst.expressions.RLike | regexp | SELECT regexp('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*') | struct<REGEXP(%SystemDrive%UsersJohn, %SystemDrive%\Users.*):boolean> |
262263
| org.apache.spark.sql.catalyst.expressions.RLike | regexp_like | SELECT regexp_like('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*') | struct<REGEXP_LIKE(%SystemDrive%UsersJohn, %SystemDrive%\Users.*):boolean> |
263264
| org.apache.spark.sql.catalyst.expressions.RLike | rlike | SELECT rlike('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*') | struct<RLIKE(%SystemDrive%UsersJohn, %SystemDrive%\Users.*):boolean> |

sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1796,3 +1796,24 @@ select try_validate_utf8(x'80')
17961796
-- !query analysis
17971797
Project [try_validate_utf8(cast(0x80 as string)) AS try_validate_utf8(X'80')#x]
17981798
+- OneRowRelation
1799+
1800+
1801+
-- !query
1802+
select quote('Spark')
1803+
-- !query analysis
1804+
Project [quote(Spark) AS quote(Spark)#x]
1805+
+- OneRowRelation
1806+
1807+
1808+
-- !query
1809+
select quote("Don't")
1810+
-- !query analysis
1811+
Project [quote(Don't) AS quote(Don't)#x]
1812+
+- OneRowRelation
1813+
1814+
1815+
-- !query
1816+
select quote(NULL)
1817+
-- !query analysis
1818+
Project [quote(cast(null as string)) AS quote(NULL)#x]
1819+
+- OneRowRelation

sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1796,3 +1796,24 @@ select try_validate_utf8(x'80')
17961796
-- !query analysis
17971797
Project [try_validate_utf8(cast(0x80 as string)) AS try_validate_utf8(X'80')#x]
17981798
+- OneRowRelation
1799+
1800+
1801+
-- !query
1802+
select quote('Spark')
1803+
-- !query analysis
1804+
Project [quote(Spark) AS quote(Spark)#x]
1805+
+- OneRowRelation
1806+
1807+
1808+
-- !query
1809+
select quote("Don't")
1810+
-- !query analysis
1811+
Project [quote(Don't) AS quote(Don't)#x]
1812+
+- OneRowRelation
1813+
1814+
1815+
-- !query
1816+
select quote(NULL)
1817+
-- !query analysis
1818+
Project [quote(cast(null as string)) AS quote(NULL)#x]
1819+
+- OneRowRelation

sql/core/src/test/resources/sql-tests/inputs/string-functions.sql

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -306,3 +306,8 @@ select validate_utf8(x'80');
306306
select try_validate_utf8('');
307307
select try_validate_utf8('abc');
308308
select try_validate_utf8(x'80');
309+
310+
-- quote
311+
select quote('Spark');
312+
select quote("Don't");
313+
select quote(NULL);

sql/core/src/test/resources/sql-tests/results/nonansi/string-functions.sql.out

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2277,3 +2277,27 @@ select try_validate_utf8(x'80')
22772277
struct<try_validate_utf8(X'80'):string>
22782278
-- !query output
22792279
NULL
2280+
2281+
2282+
-- !query
2283+
select quote('Spark')
2284+
-- !query schema
2285+
struct<quote(Spark):string>
2286+
-- !query output
2287+
'Spark'
2288+
2289+
2290+
-- !query
2291+
select quote("Don't")
2292+
-- !query schema
2293+
struct<quote(Don't):string>
2294+
-- !query output
2295+
'Don\'t'
2296+
2297+
2298+
-- !query
2299+
select quote(NULL)
2300+
-- !query schema
2301+
struct<quote(NULL):string>
2302+
-- !query output
2303+
NULL

sql/core/src/test/resources/sql-tests/results/string-functions.sql.out

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2341,3 +2341,27 @@ select try_validate_utf8(x'80')
23412341
struct<try_validate_utf8(X'80'):string>
23422342
-- !query output
23432343
NULL
2344+
2345+
2346+
-- !query
2347+
select quote('Spark')
2348+
-- !query schema
2349+
struct<quote(Spark):string>
2350+
-- !query output
2351+
'Spark'
2352+
2353+
2354+
-- !query
2355+
select quote("Don't")
2356+
-- !query schema
2357+
struct<quote(Don't):string>
2358+
-- !query output
2359+
'Don\'t'
2360+
2361+
2362+
-- !query
2363+
select quote(NULL)
2364+
-- !query schema
2365+
struct<quote(NULL):string>
2366+
-- !query output
2367+
NULL

0 commit comments

Comments
 (0)