zero323 commented on a change in pull request #31207:
URL: https://github.com/apache/spark/pull/31207#discussion_r565336841



##########
File path: python/pyspark/sql/functions.py
##########
@@ -91,13 +92,48 @@ def lit(col):
     Creates a :class:`Column` of literal value.
 
     .. versionadded:: 1.3.0
+    .. versionchanged:: 3.2.0
+        Added support for complex type literals.
 
     Examples
     --------
     >>> df.select(lit(5).alias('height')).withColumn('spark_user', 
lit(True)).take(1)
     [Row(height=5, spark_user=True)]
-    """
-    return col if isinstance(col, Column) else _invoke_function("lit", col)
+    >>> df.select(
+    ...     lit({"height": 5}).alias("data"),
+    ...     lit(["python", "scala"]).alias("languages")
+    ... ).take(1)
+    [Row(data={'height': 5}, languages=['python', 'scala'])]
+    """
+    if isinstance(col, Column):
+        return col
+
+    elif isinstance(col, list):
+        return array(*[lit(x) for x in col])
+
+    elif isinstance(col, tuple):
+        fields = (
+            # Named tuple
+            col._fields if hasattr(col, "_fields")
+            # PySpark Row
+            else col.__fields__ if hasattr(col, "__fields__")
+            # Other
+            else [f"_{i + 1}" for i in range(len(col))]
+        )
+
+        return struct(*[

Review comment:
       > Also looks like Scala side doesn't support map and struct (due to both 
the limitation of typing and the fact that `Row` doesn't hold field names, 
etc.).
   
   Actually, `typedLit` supports `Map` objects:
   
   ```scala
   scala> typedLit(Map("a" -> 1, "b" -> 2))
   res7: org.apache.spark.sql.Column = keys: [a,b], values: [1,2]
   ```
   
   and `Products`
   
   ```scala
   scala> case class FooBar(foo: Int, bar: String)
   defined class FooBar
   
   scala> typedLit(Foo(1, "a"))
   res8: org.apache.spark.sql.Column = [1,a]
   ```
   
   `Row` would be specific to PySpark, but since it is just a tuple, it seemed 
reasonable to handle it with names.
   
   > Hm, okay but these will be literals after constant folding though.
   
   The biggest difference compared to Scala is that it we handle mixed 
constants and column objects:
   
   ```python
   >>> spark.createDataFrame([(1, 2)], ("a", "b")).select(lit({"a": col("a"), 
"b": col("b")})).show()      
   +----------------+
   | map(a, a, b, b)|
   +----------------+
   |{a -> 1, b -> 2}|
   +----------------+
   ```
   
   So there is a case where `lit` results in a non-`Literal`.
   
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to