This is an automated email from the ASF dual-hosted git repository.
feiwang pushed a commit to branch branch-1.9
in repository https://gitbox.apache.org/repos/asf/kyuubi.git
The following commit(s) were added to refs/heads/branch-1.9 by this push:
new 175af0a0d7 [KYUUBI #6984] Fix ValueError when rendering MapType data
175af0a0d7 is described below
commit 175af0a0d75ed185081c0ea2076318c590f28105
Author: Reese Feng <[email protected]>
AuthorDate: Wed Mar 19 21:18:35 2025 -0700
[KYUUBI #6984] Fix ValueError when rendering MapType data
[
[KYUUBI #6984] Fix ValueError when rendering MapType data
](https://github.com/apache/kyuubi/issues/6984)
### Why are the changes needed?
The issue was caused by an incorrect iteration of MapType data in the
`%table` magic command. When iterating over a `MapType` column, the code used
`for k, v in m` directly, which leads to a `ValueError` because raw `Map`
entries may not be properly unpacked
### How was this patch tested?
- [x] Manual testing:
Executed a query with a `MapType` column and confirmed that the `%table`
command now renders it without errors.
```python
from pyspark.sql import SparkSession
from pyspark.sql.types import MapType, StringType, IntegerType
spark = SparkSession.builder \
.appName("MapFieldExample") \
.getOrCreate()
data = [
(1, {"a": "1", "b": "2"}),
(2, {"x": "10"}),
(3, {"key": "value"})
]
schema = "id INT, map_col MAP<STRING, STRING>"
df = spark.createDataFrame(data, schema=schema)
df.printSchema()
df2=df.collect()
```
using `%table` render table
```python
%table df2
```
result
```python
{'application/vnd.livy.table.v1+json': {'headers': [{'name': 'id', 'type':
'INT_TYPE'}, {'name': 'map_col', 'type': 'MAP_TYPE'}], 'data': [[1, {'a': '1',
'b': '2'}], [2, {'x': '10'}], [3, {'key': 'value'}]]}}
```
### Was this patch authored or co-authored using generative AI tooling?
No
**notice** This PR was co-authored by DeepSeek-R1.
Closes #6985 from JustFeng/patch-1.
Closes #6984
e0911ba94 [Reese Feng] Update PySparkTests for magic cmd
bc3ce1a49 [Reese Feng] Update PySparkTests for magic cmd
200d7ad9b [Reese Feng] Fix syntax error in dict iteration in
magic_table_convert_map
Authored-by: Reese Feng <[email protected]>
Signed-off-by: Wang, Fei <[email protected]>
(cherry picked from commit a54ee39ab338e310c6b9a508ad8f14c0bd82fa0f)
Signed-off-by: Wang, Fei <[email protected]>
---
.../src/main/resources/python/execute_python.py | 2 +-
.../scala/org/apache/kyuubi/engine/spark/PySparkTests.scala | 11 +++++++----
2 files changed, 8 insertions(+), 5 deletions(-)
diff --git
a/externals/kyuubi-spark-sql-engine/src/main/resources/python/execute_python.py
b/externals/kyuubi-spark-sql-engine/src/main/resources/python/execute_python.py
index 3e7f3caa25..1fff10a676 100644
---
a/externals/kyuubi-spark-sql-engine/src/main/resources/python/execute_python.py
+++
b/externals/kyuubi-spark-sql-engine/src/main/resources/python/execute_python.py
@@ -314,7 +314,7 @@ def magic_table_convert_map(m):
last_value_type = None
converted_items = {}
- for key, value in m:
+ for key, value in m.items():
key_type, key = magic_table_convert(key)
value_type, value = magic_table_convert(value)
diff --git
a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/spark/PySparkTests.scala
b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/spark/PySparkTests.scala
index 6ec0b87f1d..8a1208d8a2 100644
---
a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/spark/PySparkTests.scala
+++
b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/spark/PySparkTests.scala
@@ -137,22 +137,25 @@ class PySparkTests extends WithKyuubiServer with
HiveJDBCTestHelper {
withSessionConf()(Map(KyuubiConf.ENGINE_SPARK_PYTHON_MAGIC_ENABLED.key ->
"true"))() {
withMultipleConnectionJdbcStatement()({ stmt =>
val statement = stmt.asInstanceOf[KyuubiStatement]
- statement.executePython("x = [[1, 'a'], [3, 'b']]")
+ statement.executePython("x = [[1, 'a', {'k1':'v1'}], [3, 'b',
{'k2':'v2'}]]")
val resultSet1 = statement.executePython("%json x")
assert(resultSet1.next())
val output1 = resultSet1.getString("output")
- assert(output1 == "{\"application/json\":[[1,\"a\"],[3,\"b\"]]}")
+ assert(output1 ==
"""{"application/json":[[1,"a",{"k1":"v1"}],[3,"b",{"k2":"v2"}]]}""")
val resultSet2 = statement.executePython("%table x")
assert(resultSet2.next())
val output2 = resultSet2.getString("output")
assert(output2 == "{\"application/vnd.livy.table.v1+json\":{" +
"\"headers\":[" +
-
"{\"name\":\"0\",\"type\":\"INT_TYPE\"},{\"name\":\"1\",\"type\":\"STRING_TYPE\"}"
+
+ "{\"name\":\"0\",\"type\":\"INT_TYPE\"}," +
+ "{\"name\":\"1\",\"type\":\"STRING_TYPE\"}," +
+ "{\"name\":\"2\",\"type\":\"MAP_TYPE\"}" +
"]," +
"\"data\":[" +
- "[1,\"a\"],[3,\"b\"]" +
+ "[1,\"a\",{\"k1\":\"v1\"}]," +
+ "[3,\"b\",{\"k2\":\"v2\"}]" +
"]}}")
Seq("table", "json", "matplot").foreach { magic =>