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 3e7f3caa2..1fff10a67 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 6ec0b87f1..8a1208d8a 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 =>