Skip to content

Commit e10b874

Browse files
viiryacloud-fan
authored andcommitted
[SPARK-16622][SQL] Fix NullPointerException when the returned value of the called method in Invoke is null
## What changes were proposed in this pull request? Currently we don't check the value returned by called method in `Invoke`. When the returned value is null and is assigned to a variable of primitive type, `NullPointerException` will be thrown. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14259 from viirya/agg-empty-ds.
1 parent 47f5b88 commit e10b874

File tree

2 files changed

+65
-17
lines changed

2 files changed

+65
-17
lines changed

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

Lines changed: 30 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -134,44 +134,57 @@ case class Invoke(
134134
val argGen = arguments.map(_.genCode(ctx))
135135
val argString = argGen.map(_.value).mkString(", ")
136136

137-
val callFunc = if (method.isDefined && method.get.getReturnType.isPrimitive) {
138-
s"${obj.value}.$functionName($argString)"
139-
} else {
140-
s"(${ctx.boxedType(javaType)}) ${obj.value}.$functionName($argString)"
141-
}
137+
val returnPrimitive = method.isDefined && method.get.getReturnType.isPrimitive
138+
val needTryCatch = method.isDefined && method.get.getExceptionTypes.nonEmpty
142139

143-
val setIsNull = if (propagateNull && arguments.nonEmpty) {
144-
s"boolean ${ev.isNull} = ${obj.isNull} || ${argGen.map(_.isNull).mkString(" || ")};"
140+
def getFuncResult(resultVal: String, funcCall: String): String = if (needTryCatch) {
141+
s"""
142+
try {
143+
$resultVal = $funcCall;
144+
} catch (Exception e) {
145+
org.apache.spark.unsafe.Platform.throwException(e);
146+
}
147+
"""
145148
} else {
146-
s"boolean ${ev.isNull} = ${obj.isNull};"
149+
s"$resultVal = $funcCall;"
147150
}
148151

149-
val evaluate = if (method.forall(_.getExceptionTypes.isEmpty)) {
150-
s"final $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : $callFunc;"
152+
val evaluate = if (returnPrimitive) {
153+
getFuncResult(ev.value, s"${obj.value}.$functionName($argString)")
151154
} else {
155+
val funcResult = ctx.freshName("funcResult")
152156
s"""
153-
$javaType ${ev.value} = ${ctx.defaultValue(javaType)};
154-
try {
155-
${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(javaType)} : $callFunc;
156-
} catch (Exception e) {
157-
org.apache.spark.unsafe.Platform.throwException(e);
157+
Object $funcResult = null;
158+
${getFuncResult(funcResult, s"${obj.value}.$functionName($argString)")}
159+
if ($funcResult == null) {
160+
${ev.isNull} = true;
161+
} else {
162+
${ev.value} = (${ctx.boxedType(javaType)}) $funcResult;
158163
}
159164
"""
160165
}
161166

167+
val setIsNull = if (propagateNull && arguments.nonEmpty) {
168+
s"boolean ${ev.isNull} = ${obj.isNull} || ${argGen.map(_.isNull).mkString(" || ")};"
169+
} else {
170+
s"boolean ${ev.isNull} = ${obj.isNull};"
171+
}
172+
162173
// If the function can return null, we do an extra check to make sure our null bit is still set
163174
// correctly.
164175
val postNullCheck = if (ctx.defaultValue(dataType) == "null") {
165176
s"${ev.isNull} = ${ev.value} == null;"
166177
} else {
167178
""
168179
}
169-
170180
val code = s"""
171181
${obj.code}
172182
${argGen.map(_.code).mkString("\n")}
173183
$setIsNull
174-
$evaluate
184+
$javaType ${ev.value} = ${ctx.defaultValue(dataType)};
185+
if (!${ev.isNull}) {
186+
$evaluate
187+
}
175188
$postNullCheck
176189
"""
177190
ev.copy(code = code)
Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,35 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst.expressions
19+
20+
import org.apache.spark.SparkFunSuite
21+
import org.apache.spark.sql.catalyst.InternalRow
22+
import org.apache.spark.sql.catalyst.expressions.objects.Invoke
23+
import org.apache.spark.sql.types.{IntegerType, ObjectType}
24+
25+
26+
class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
27+
28+
test("SPARK-16622: The returned value of the called method in Invoke can be null") {
29+
val inputRow = InternalRow.fromSeq(Seq((false, null)))
30+
val cls = classOf[Tuple2[Boolean, java.lang.Integer]]
31+
val inputObject = BoundReference(0, ObjectType(cls), nullable = true)
32+
val invoke = Invoke(inputObject, "_2", IntegerType)
33+
checkEvaluationWithGeneratedMutableProjection(invoke, null, inputRow)
34+
}
35+
}

0 commit comments

Comments
 (0)