[SPARK-26071][FOLLOWUP] Improve migration guide of disallowing map type map key
### What changes were proposed in this pull request? mention the workaround if users do want to use map type as key, and add a test to demonstrate it. ### Why are the changes needed? it's better to provide an alternative when we ban something. ### Does this PR introduce any user-facing change? no ### How was this patch tested? N/A Closes #27621 from cloud-fan/map. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
parent
4248b7fbb9
commit
704d249a56
|
@ -43,7 +43,7 @@ license: |
|
|||
|
||||
- The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set.
|
||||
|
||||
- In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful.
|
||||
- In Spark version 2.4 and earlier, users can create map values with map type key via built-in function such as `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can use `map_entries` function to convert map to array<struct<key, value>> as a workaround. In addition, users can still read map values with map type key from data source or Java/Scala collections, though it is discouraged.
|
||||
|
||||
- In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`.
|
||||
|
||||
|
|
|
@ -3584,6 +3584,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("SPARK-26071: convert map to array and use as map key") {
|
||||
val df = Seq(Map(1 -> "a")).toDF("m")
|
||||
intercept[AnalysisException](df.select(map($"m", lit(1))))
|
||||
checkAnswer(
|
||||
df.select(map(map_entries($"m"), lit(1))),
|
||||
Row(Map(Seq(Row(1, "a")) -> 1)))
|
||||
}
|
||||
}
|
||||
|
||||
object DataFrameFunctionsSuite {
|
||||
|
|
Loading…
Reference in a new issue