Skip to content

Commit b2bfb33

Browse files
committed
address comments
1 parent 9df6274 commit b2bfb33

File tree

7 files changed

+32
-47
lines changed

7 files changed

+32
-47
lines changed

docs/sql-migration-guide-upgrade.md

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,15 @@ displayTitle: Spark SQL Upgrading Guide
1717

1818
- Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`.
1919

20-
- In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independetly of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`.
20+
- In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independetly of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`.
2121

2222
- The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set.
2323

2424
- 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.
25-
25+
2626
- 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`.
2727

28-
- In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy.
28+
- In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be udefined.
2929

3030
## Upgrading From Spark SQL 2.3 to 2.4
3131

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala

Lines changed: 2 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -549,7 +549,7 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres
549549
private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType)
550550

551551
override def eval(input: InternalRow): Any = {
552-
val maps = children.map(_.eval(input).asInstanceOf[MapData]).toArray
552+
val maps = children.map(_.eval(input).asInstanceOf[MapData])
553553
if (maps.contains(null)) {
554554
return null
555555
}
@@ -561,12 +561,8 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres
561561
s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.")
562562
}
563563

564-
mapBuilder.reset()
565-
var i = 0
566-
while (i < maps.length) {
567-
val map = maps(i)
564+
for (map <- maps) {
568565
mapBuilder.putAll(map.keyArray(), map.valueArray())
569-
i += 1
570566
}
571567
mapBuilder.build()
572568
}
@@ -648,7 +644,6 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres
648644
|}
649645
|ArrayData $finKeysName = $keyConcat($keyArgsName, (int) $numElementsName);
650646
|ArrayData $finValsName = $valueConcat($valArgsName, (int) $numElementsName);
651-
|$builderTerm.reset();
652647
|${ev.value} = $builderTerm.from($finKeysName, $finValsName);
653648
""".stripMargin
654649

@@ -752,7 +747,6 @@ case class MapFromEntries(child: Expression) extends UnaryExpression {
752747
}
753748
}
754749

755-
mapBuilder.reset()
756750
i = 0
757751
while (i < numEntries) {
758752
mapBuilder.put(entries.getStruct(i, 2))
@@ -769,7 +763,6 @@ case class MapFromEntries(child: Expression) extends UnaryExpression {
769763
ctx.nullArrayElementsSaveExec(nullEntries, ev.isNull, c) {
770764
s"""
771765
|final int $numEntries = $c.numElements();
772-
|$builderTerm.reset();
773766
|for (int $i = 0; $i < $numEntries; $i++) {
774767
| $builderTerm.put($c.getStruct($i, 2));
775768
|}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala

Lines changed: 1 addition & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -169,7 +169,6 @@ case class CreateMap(children: Seq[Expression]) extends Expression {
169169
private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType)
170170

171171
override def eval(input: InternalRow): Any = {
172-
mapBuilder.reset()
173172
var i = 0
174173
while (i < keys.length) {
175174
mapBuilder.put(keys(i).eval(input), values(i).eval(input))
@@ -191,7 +190,6 @@ case class CreateMap(children: Seq[Expression]) extends Expression {
191190
$assignKeys
192191
$allocationValueData
193192
$assignValues
194-
$builderTerm.reset();
195193
final MapData ${ev.value} = $builderTerm.from($keyArrayData, $valueArrayData);
196194
"""
197195
ev.copy(code = code, isNull = FalseLiteral)
@@ -239,17 +237,13 @@ case class MapFromArrays(left: Expression, right: Expression)
239237
override def nullSafeEval(keyArray: Any, valueArray: Any): Any = {
240238
val keyArrayData = keyArray.asInstanceOf[ArrayData]
241239
val valueArrayData = valueArray.asInstanceOf[ArrayData]
242-
mapBuilder.reset()
243240
mapBuilder.from(keyArrayData.copy(), valueArrayData.copy())
244241
}
245242

246243
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
247244
nullSafeCodeGen(ctx, ev, (keyArrayData, valueArrayData) => {
248245
val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder)
249-
s"""
250-
|$builderTerm.reset();
251-
|${ev.value} = $builderTerm.from($keyArrayData.copy(), $valueArrayData.copy());
252-
""".stripMargin
246+
s"${ev.value} = $builderTerm.from($keyArrayData.copy(), $valueArrayData.copy());"
253247
})
254248
}
255249

@@ -467,7 +461,6 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
467461
inputString.asInstanceOf[UTF8String].split(stringDelimiter.asInstanceOf[UTF8String], -1)
468462
val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String]
469463

470-
mapBuilder.reset()
471464
var i = 0
472465
while (i < keyValues.length) {
473466
val keyValueArray = keyValues(i).split(keyValueDelimiterUTF8String, 2)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -538,7 +538,6 @@ case class TransformKeys(
538538
resultKeys.update(i, result)
539539
i += 1
540540
}
541-
mapBuilder.reset()
542541
mapBuilder.from(resultKeys, map.valueArray())
543542
}
544543

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala

Lines changed: 22 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,15 @@ package org.apache.spark.sql.catalyst.util
2020
import scala.collection.mutable
2121

2222
import org.apache.spark.sql.catalyst.InternalRow
23-
import org.apache.spark.sql.types.{AtomicType, CalendarIntervalType, DataType, MapType}
23+
import org.apache.spark.sql.types._
2424

2525
/**
2626
* A builder of [[ArrayBasedMapData]], which fails if a null map key is detected, and removes
2727
* duplicated map keys w.r.t. the last wins policy.
2828
*/
2929
class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Serializable {
3030
assert(!keyType.existsRecursively(_.isInstanceOf[MapType]), "key of map cannot be/contain map")
31+
assert(keyType != NullType, "map key cannot be null type.")
3132

3233
private lazy val keyToIndex = keyType match {
3334
case _: AtomicType | _: CalendarIntervalType => mutable.HashMap.empty[Any, Int]
@@ -44,12 +45,6 @@ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Seria
4445
private lazy val keyGetter = InternalRow.getAccessor(keyType)
4546
private lazy val valueGetter = InternalRow.getAccessor(valueType)
4647

47-
def reset(): Unit = {
48-
keyToIndex.clear()
49-
keys.clear()
50-
values.clear()
51-
}
52-
5348
def put(key: Any, value: Any): Unit = {
5449
if (key == null) {
5550
throw new RuntimeException("Cannot use null as map key.")
@@ -74,19 +69,6 @@ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Seria
7469
put(keyGetter(entry, 0), valueGetter(entry, 1))
7570
}
7671

77-
def putAll(keyArray: Array[Any], valueArray: Array[Any]): Unit = {
78-
if (keyArray.length != valueArray.length) {
79-
throw new RuntimeException(
80-
"The key array and value array of MapData must have the same length.")
81-
}
82-
83-
var i = 0
84-
while (i < keyArray.length) {
85-
put(keyArray(i), valueArray(i))
86-
i += 1
87-
}
88-
}
89-
9072
def putAll(keyArray: ArrayData, valueArray: ArrayData): Unit = {
9173
if (keyArray.numElements() != valueArray.numElements()) {
9274
throw new RuntimeException(
@@ -100,16 +82,34 @@ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Seria
10082
}
10183
}
10284

85+
private def reset(): Unit = {
86+
keyToIndex.clear()
87+
keys.clear()
88+
values.clear()
89+
}
90+
91+
/**
92+
* Builds the result [[ArrayBasedMapData]] and reset this builder to free up the resources. The
93+
* builder becomes fresh afterward and is ready to take input and build another map.
94+
*/
10395
def build(): ArrayBasedMapData = {
104-
new ArrayBasedMapData(new GenericArrayData(keys.toArray), new GenericArrayData(values.toArray))
96+
val map = new ArrayBasedMapData(
97+
new GenericArrayData(keys.toArray), new GenericArrayData(values.toArray))
98+
reset()
99+
map
105100
}
106101

102+
/**
103+
* Builds a [[ArrayBasedMapData]] from the given key and value array and reset this builder. The
104+
* builder becomes fresh afterward and is ready to take input and build another map.
105+
*/
107106
def from(keyArray: ArrayData, valueArray: ArrayData): ArrayBasedMapData = {
108-
assert(keyToIndex.isEmpty, "'from' can only be called with a fresh GenericMapBuilder.")
107+
assert(keyToIndex.isEmpty, "'from' can only be called with a fresh ArrayBasedMapBuilder.")
109108
putAll(keyArray, valueArray)
110109
if (keyToIndex.size == keyArray.numElements()) {
111110
// If there is no duplicated map keys, creates the MapData with the input key and value array,
112111
// as they might already in unsafe format and are more efficient.
112+
reset()
113113
new ArrayBasedMapData(keyArray, valueArray)
114114
} else {
115115
build()

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -29,11 +29,10 @@ class ArrayBasedMapBuilderSuite extends SparkFunSuite {
2929
val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType)
3030
builder.put(1, 1)
3131
builder.put(InternalRow(2, 2))
32-
builder.putAll(Array(3: Any), Array(3: Any))
33-
builder.putAll(new GenericArrayData(Seq(4)), new GenericArrayData(Seq(4)))
32+
builder.putAll(new GenericArrayData(Seq(3)), new GenericArrayData(Seq(3)))
3433
val map = builder.build()
35-
assert(map.numElements() == 4)
36-
assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4))
34+
assert(map.numElements() == 3)
35+
assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 1, 2 -> 2, 3 -> 3))
3736
}
3837

3938
test("fail with null key") {

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -561,6 +561,7 @@ private[parquet] class ParquetRowConverter(
561561
override def end(): Unit = {
562562
// The parquet map may contains null or duplicated map keys. When it happens, the behavior is
563563
// undefined.
564+
// TODO (SPARK-26174): disallow it with a config.
564565
updater.set(ArrayBasedMapData(currentKeys.toArray, currentValues.toArray))
565566
}
566567

0 commit comments

Comments
 (0)