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-48665][PYTHON][CONNECT] Support providing a dict in pyspark lit to create a map. #49318

Open
wants to merge 22 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 11 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions python/pyspark/errors/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,11 @@
"The value <provider> does not represent a correct collation provider. Supported providers are: [<supportedProviders>]."
]
},
"COLUMN_IN_DICT" : {
"message": [
"`<func_name>` does not allow a Column in a dict."
]
},
"COLUMN_IN_LIST": {
"message": [
"`<func_name>` does not allow a Column in a list."
Expand Down
13 changes: 12 additions & 1 deletion python/pyspark/sql/connect/functions/builtin.py
Original file line number Diff line number Diff line change
Expand Up @@ -257,8 +257,9 @@ def col(col: str) -> Column:
column = col


def lit(col: Any) -> Column:
def lit(col: Any, to_struct: bool = False) -> Column:
skanderboudawara marked this conversation as resolved.
Show resolved Hide resolved
from pyspark.sql.connect.column import Column as ConnectColumn
from itertools import chain

if isinstance(col, Column):
return col
Expand All @@ -276,6 +277,16 @@ def lit(col: Any) -> Column:
messageParameters={"dtype": col.dtype.name},
)
return array(*[lit(c) for c in col]).cast(ArrayType(dt))
elif isinstance(col, dict):
if any(isinstance(c, Column) for c in col.values()):
raise PySparkValueError(
errorClass="COLUMN_IN_DICT", messageParameters={"func_name": "lit"}
)
# Convert to struct or map based on the parameter `to_struct`
if to_struct:
return struct(*[lit(value).alias(key) for key, value in col.items()])
skanderboudawara marked this conversation as resolved.
Show resolved Hide resolved
else:
return create_map(*[lit(x) for x in chain(*col.items())])
return ConnectColumn(LiteralExpression._from_value(col))


Expand Down
50 changes: 46 additions & 4 deletions python/pyspark/sql/functions/builtin.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
Union,
ValuesView,
)
from itertools import chain

from pyspark.errors import PySparkTypeError, PySparkValueError
from pyspark.errors.utils import _with_origin
Expand Down Expand Up @@ -153,7 +154,7 @@ def _options_to_str(options: Optional[Mapping[str, Any]] = None) -> Mapping[str,


@_try_remote_functions
def lit(col: Any) -> Column:
def lit(col: Any, to_struct: bool = False) -> Column:
"""
Creates a :class:`~pyspark.sql.Column` of literal value.

Expand All @@ -164,12 +165,17 @@ def lit(col: Any) -> Column:

Parameters
----------
col : :class:`~pyspark.sql.Column`, str, int, float, bool or list, NumPy literals or ndarray.
the value to make it as a PySpark literal. If a column is passed,
col : :class:`~pyspark.sql.Column`, str, int, float, bool or list, NumPy literals, ndarray
or dict. the value to make it as a PySpark literal. If a column is passed,
it returns the column as is.

.. versionchanged:: 3.4.0
to_struct: bool, optional, default False
If True, the column will be converted to a struct column. If False, the column will be
converted to a map column. Default is False. only has an effect when col is a dict.

.. versionchanged:: 4.0.0
Since 3.4.0, it supports the list type.
Since 4.0.0, it supports the dict type for the creation of a map.

Returns
-------
Expand Down Expand Up @@ -253,6 +259,30 @@ def lit(col: Any) -> Column:
+------------------+-------+-----------------+--------------------+
| [true, false]| []| [1.5, 0.1]| [a, b, c]|
+------------------+-------+-----------------+--------------------+

Example 7: Creating a literal column as a map from a dict.

>>> import pyspark.sql.functions as sf
>>> spark.range(1).select(
... sf.lit({"a": 1, "b": 2}).alias("map_col")
... ).show() # doctest: +SKIP
skanderboudawara marked this conversation as resolved.
Show resolved Hide resolved
+----------------+
| map_col|
+----------------+
|{a -> 1, b -> 2}|
+----------------+

Example 8: Creating a literal column as a struct from a dict.

>>> import pyspark.sql.functions as sf
>>> spark.range(1).select(
... sf.lit({"a": 1, "b": 2}, true).alias("struct_col")
... ).show() # doctest: +SKIP
+----------------+
| struct_col|
+----------------+
|{a -> 1, b -> 2}|
+----------------+
"""
if isinstance(col, Column):
return col
Expand All @@ -262,6 +292,18 @@ def lit(col: Any) -> Column:
errorClass="COLUMN_IN_LIST", messageParameters={"func_name": "lit"}
)
return array(*[lit(item) for item in col])
elif isinstance(col, dict):
skanderboudawara marked this conversation as resolved.
Show resolved Hide resolved
# Skip checking if the keys are column as Columns are not hashable
# and cannot be used as dict keys in the first place.
if any(isinstance(value, Column) for value in col.values()):
raise PySparkValueError(
errorClass="COLUMN_IN_DICT", messageParameters={"func_name": "lit"}
)
# Convert to struct or map based on the parameter `to_struct`
if to_struct:
return struct(*[lit(value).alias(key) for key, value in col.items()])
else:
return create_map(*[lit(x) for x in chain(*col.items())])
elif _has_numpy:
if isinstance(col, np.generic):
dt = _from_numpy_type(col.dtype)
Expand Down
38 changes: 38 additions & 0 deletions python/pyspark/sql/tests/test_functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -1316,6 +1316,44 @@ def test_lit_list(self):
messageParameters={"func_name": "lit"},
)

# SPARK-48665: added support for dict type
def test_lit_dict(self):
test_dict = {"a": 1, "b": 2}
actual = self.spark.range(1).select(F.lit(test_dict, to_struct=True)).first()[0]
# Convert struct return to dict
actual = actual.asDict()
self.assertEqual(actual, test_dict)

test_dict = {"a": 1, "b": 2}
actual = self.spark.range(1).select(F.lit(test_dict)).first()[0]
self.assertEqual(actual, test_dict)

test_dict = {"a": {"1": 1}, "b": {"2": 2}}
actual = self.spark.range(1).select(F.lit(test_dict)).first()[0]
self.assertEqual(actual, test_dict)

with self.sql_conf({"spark.sql.ansi.enabled": False}):
test_dict = {"a": 1, "b": "2", "c": None}
expected_dict = {"a": "1", "b": "2", "c": None}
actual = self.spark.range(1).select(F.lit(test_dict)).first()[0]
self.assertEqual(actual, expected_dict)

df = self.spark.range(10)
dicts = [
{"a": df.id},
{"a": {"b": df.id}},
]

for d in dicts:
with self.assertRaises(PySparkValueError) as pe:
F.lit(d)

self.check_error(
exception=pe.exception,
errorClass="COLUMN_IN_DICT",
messageParameters={"func_name": "lit"},
)

# Test added for SPARK-39832; change Python API to accept both col & str as input
def test_regexp_replace(self):
df = self.spark.createDataFrame(
Expand Down
Loading