Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -3404,8 +3404,10 @@ class PlanGenerationTestSuite
// Handle parameterized scala types e.g.: List, Seq and Map.
fn.typedLit(Some(1)),
fn.typedLit(Array(1, 2, 3)),
fn.typedLit[Array[Integer]](Array(null, null)),
fn.typedLit(Seq(1, 2, 3)),
fn.typedLit(Map("a" -> 1, "b" -> 2)),
fn.typedLit(mutable.LinkedHashMap("a" -> 1, "b" -> 2)),
fn.typedLit(mutable.LinkedHashMap[String, Integer]("a" -> null, "b" -> null)),
fn.typedLit(("a", 2, 1.0)),
fn.typedLit[Option[Int]](None),
fn.typedLit[Array[Option[Int]]](Array(Some(1))),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1681,6 +1681,13 @@ class ClientE2ETestSuite
assert(df.count() == 100)
}
}

test("SPARK-53553: null value handling in literals") {
val df = spark.sql("select 1").select(typedlit(Array[Integer](1, null)).as("arr_col"))
val result = df.collect()
assert(result.length === 1)
assert(result(0).getAs[Array[Integer]]("arr_col") === Array(1, null))
}
}

private[sql] case class ClassData(a: String, b: Int)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,14 @@ object LiteralValueProtoConverter {
}

(literal, dataType) match {
case (v: Option[_], _: DataType) =>
if (v.isDefined) {
toLiteralProtoBuilder(v.get)
} else {
builder.setNull(toConnectProtoType(dataType))
}
case (null, _) =>
builder.setNull(toConnectProtoType(dataType))
case (v: mutable.ArraySeq[_], ArrayType(_, _)) =>
toLiteralProtoBuilder(v.array, dataType)
case (v: immutable.ArraySeq[_], ArrayType(_, _)) =>
Expand All @@ -175,12 +183,6 @@ object LiteralValueProtoConverter {
builder.setMap(mapBuilder(v, keyType, valueType))
case (v, structType: StructType) =>
builder.setStruct(structBuilder(v, structType))
case (v: Option[_], _: DataType) =>
if (v.isDefined) {
toLiteralProtoBuilder(v.get)
} else {
builder.setNull(toConnectProtoType(dataType))
}
case _ => toLiteralProtoBuilder(literal)
}
}
Expand Down Expand Up @@ -297,7 +299,7 @@ object LiteralValueProtoConverter {
}

private def getScalaConverter(dataType: proto.DataType): proto.Expression.Literal => Any = {
if (dataType.hasShort) { v =>
val converter: proto.Expression.Literal => Any = if (dataType.hasShort) { v =>
v.getShort.toShort
} else if (dataType.hasInteger) { v =>
v.getInteger
Expand Down Expand Up @@ -339,6 +341,7 @@ object LiteralValueProtoConverter {
} else {
throw InvalidPlanInput(s"Unsupported Literal Type: $dataType)")
}
v => if (v.hasNull) null else converter(v)
}

def toCatalystArray(array: proto.Expression.Literal.Array): Array[_] = {
Expand Down
Original file line number Diff line number Diff line change
@@ -1,2 +1,2 @@
Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, ... 19 more fields]
Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, ... 21 more fields]
+- LocalRelation <empty>, [id#0L, a#0, b#0]
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,8 @@
}, {
"literal": {
"null": {
"null": {
"string": {
"collation": "UTF8_BINARY"
}
}
},
Expand Down Expand Up @@ -814,6 +815,43 @@
}
}
}
}, {
"literal": {
"array": {
"elementType": {
"integer": {
}
},
"elements": [{
"null": {
"integer": {
}
}
}, {
"null": {
"integer": {
}
}
}]
}
},
"common": {
"origin": {
"jvmOrigin": {
"stackTrace": [{
"classLoaderName": "app",
"declaringClass": "org.apache.spark.sql.functions$",
"methodName": "typedLit",
"fileName": "functions.scala"
}, {
"classLoaderName": "app",
"declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
"methodName": "~~trimmed~anonfun~~",
"fileName": "PlanGenerationTestSuite.scala"
}]
}
}
}
}, {
"literal": {
"array": {
Expand Down Expand Up @@ -888,6 +926,53 @@
}
}
}
}, {
"literal": {
"map": {
"keyType": {
"string": {
"collation": "UTF8_BINARY"
}
},
"valueType": {
"integer": {
}
},
"keys": [{
"string": "a"
}, {
"string": "b"
}],
"values": [{
"null": {
"integer": {
}
}
}, {
"null": {
"integer": {
}
}
}]
}
},
"common": {
"origin": {
"jvmOrigin": {
"stackTrace": [{
"classLoaderName": "app",
"declaringClass": "org.apache.spark.sql.functions$",
"methodName": "typedLit",
"fileName": "functions.scala"
}, {
"classLoaderName": "app",
"declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
"methodName": "~~trimmed~anonfun~~",
"fileName": "PlanGenerationTestSuite.scala"
}]
}
}
}
}, {
"literal": {
"struct": {
Expand Down
Binary file not shown.