You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "koert kuipers (Jira)" <ji...@apache.org> on 2021/11/27 21:48:00 UTC

[jira] [Created] (SPARK-37476) udaf doesnt work with nullable case class result

koert kuipers created SPARK-37476:
-------------------------------------

             Summary: udaf doesnt work with nullable case class result 
                 Key: SPARK-37476
                 URL: https://issues.apache.org/jira/browse/SPARK-37476
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 3.2.0
         Environment: spark master branch on nov 27
            Reporter: koert kuipers


i have a need to have an aggregation return a nullable case class. there seems to be no way to get this to work. the suggestion to wrap the result in an option doesnt work either.

first attempt using nulls:

 
{code:java}
val sumAndProductAgg = new Aggregator[Double, SumAndProduct, SumAndProduct] {
  def zero: SumAndProduct = null
  def reduce(b: SumAndProduct, a: Double): SumAndProduct =
    if (b == null) {
      SumAndProduct(a, a)
    } else {
      SumAndProduct(b.sum + a, b.product * a)
    }
  def merge(b1: SumAndProduct, b2: SumAndProduct): SumAndProduct =
    if (b1 == null) {
      b2
    } else if (b2 == null) {
      b1
    } else {
      SumAndProduct(b1.sum + b2.sum, b1.product * b2.product)
    }
  def finish(r: SumAndProduct): SumAndProduct = r
  def bufferEncoder: Encoder[SumAndProduct] = ExpressionEncoder()
  def outputEncoder: Encoder[SumAndProduct] = ExpressionEncoder()
}

val df = Seq.empty[Double]
  .toDF()
  .select(udaf(sumAndProductAgg).apply(col("value")))
df.printSchema()
df.show()
{code}
this gives:

 
{code:java}
root
 |-- $anon$3(value): struct (nullable = true)
 |    |-- sum: double (nullable = false)
 |    |-- product: double (nullable = false)
16:44:54.882 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1491.0 (TID 1929)
java.lang.RuntimeException: Error while encoding: java.lang.NullPointerException: Null value appeared in non-nullable field:
top level Product or row object
If the schema is inferred from a Scala tuple/case class, or a Java bean, please try to use scala.Option[_] or other nullable types (e.g. java.lang.Integer instead of int/scala.Int).
knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).sum AS sum#20070
knownnotnull(assertnotnull(input[0, org.apache.spark.sql.SumAndProduct, true])).product AS product#20071
    at org.apache.spark.sql.errors.QueryExecutionErrors$.expressionEncodingError(QueryExecutionErrors.scala:1125)
 {code}
taking the advice to heart and using option we get to the second attempt using options:

 

 
{code:java}
val sumAndProductAgg = new Aggregator[Double, Option[SumAndProduct], Option[SumAndProduct]] {
  def zero: Option[SumAndProduct] = None
  def reduce(b: Option[SumAndProduct], a: Double): Option[SumAndProduct] =
    b
      .map{ b => SumAndProduct(b.sum + a, b.product * a) }
      .orElse{ Option(SumAndProduct(a, a)) }
  def merge(b1: Option[SumAndProduct], b2: Option[SumAndProduct]): Option[SumAndProduct] =
    b1.map{ b1 =>
      b2.map{ b2 =>
        SumAndProduct(b1.sum + b2.sum, b1.product * b2.product)
      }.getOrElse(b1)
    }.orElse(b2)
  def finish(r: Option[SumAndProduct]): Option[SumAndProduct] = r
  def bufferEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder()
  def outputEncoder: Encoder[Option[SumAndProduct]] = ExpressionEncoder()
}
val df = Seq.empty[Double]
  .toDF()
  .select(udaf(sumAndProductAgg).apply(col("value")))
df.printSchema()
df.show()
{code}
this gives:

 

 
{code:java}
root
 |-- $anon$4(value): struct (nullable = true)
 |    |-- sum: double (nullable = false)
 |    |-- product: double (nullable = false)
16:44:54.998 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1493.0 (TID 1930)
java.lang.AssertionError: index (1) should < 1
    at org.apache.spark.sql.catalyst.expressions.UnsafeRow.assertIndexIsValid(UnsafeRow.java:142)
    at org.apache.spark.sql.catalyst.expressions.UnsafeRow.isNullAt(UnsafeRow.java:338)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
    at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:260)
    at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:107)
{code}
 

 

 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org