Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/spark_expressions_support.md
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@
- [x] map_contains_key
- [ ] map_entries
- [ ] map_from_arrays
- [ ] map_from_entries
- [x] map_from_entries
- [x] map_keys
- [ ] map_values
- [ ] str_to_map
Expand Down
8 changes: 8 additions & 0 deletions spark/src/main/scala/org/apache/comet/serde/maps.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.comet.serde

import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._

import org.apache.comet.serde.QueryPlanSerde.{createBinaryExpr, exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto}
Expand Down Expand Up @@ -136,6 +137,8 @@ object CometMapContainsKey extends CometExpressionSerde[MapContainsKey] {
object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") {
val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries"
val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries"
val lastWinUnsupportedReason =
"spark.sql.mapKeyDedupPolicy=LAST_WIN is not yet supported natively for map_from_entries"

private def containsBinary(dataType: DataType): Boolean = {
dataType match {
Expand All @@ -153,6 +156,11 @@ object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from
if (containsBinary(expr.dataType.valueType)) {
return Incompatible(Some(valueUnsupportedReason))
}
// Only the default EXCEPTION policy is supported natively; fall back otherwise.
if (!SQLConf.get.getConfString("spark.sql.mapKeyDedupPolicy", "EXCEPTION")
.equalsIgnoreCase("EXCEPTION")) {
return Incompatible(Some(lastWinUnsupportedReason))
}
Compatible(None)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ package org.apache.comet
import scala.util.Random

import org.apache.hadoop.fs.Path
import org.apache.spark.SparkException
import org.apache.spark.sql.CometTestBase
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -236,4 +237,14 @@ class CometMapExpressionSuite extends CometTestBase {
}
}

test("map_from_entries - duplicate keys throw under default EXCEPTION policy") {
val df = sql("select map_from_entries(array(struct(1, 'a'), struct(1, 'b')))")
val ex = intercept[SparkException] {
df.collect()
}
assert(
ex.getMessage.contains("DUPLICATED_MAP_KEY"),
s"expected DUPLICATED_MAP_KEY error class, got: ${ex.getMessage}")
}

}