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/12/01 16:58:00 UTC

[jira] [Comment Edited] (SPARK-37476) udaf doesnt work with nullable (or option of) case class result

    [ https://issues.apache.org/jira/browse/SPARK-37476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17451921#comment-17451921 ] 

koert kuipers edited comment on SPARK-37476 at 12/1/21, 4:57 PM:
-----------------------------------------------------------------

yes it works well
{code:java}
import java.lang.{Double => JDouble}

val sumAgg = new Aggregator[Double, JDouble, JDouble] {
  def zero: JDouble = null

  def reduce(b: JDouble, a: Double): JDouble =
    if (b == null) {
      a
    } else {
      b + a
    }

  def merge(b1: JDouble, b2: JDouble): JDouble =
    if (b1 == null) {
      b2
    } else if (b2 == null) {
      b1
    } else {
      b1 + b2
    }

  def finish(r: JDouble): JDouble = r

  def bufferEncoder: Encoder[JDouble] = ExpressionEncoder()
  def outputEncoder: Encoder[JDouble] = ExpressionEncoder()
}

val df = Seq.empty[Double]
  .toDF()
  .select(udaf(sumAgg).apply(col("value")))

df.printSchema()
root
 |-- $anon$3(value): double (nullable = true) 

df.show()
+--------------+
|$anon$3(value)|
+--------------+
|          null|
+--------------+
 {code}
it works with Option without issues too

 


was (Author: koert):
yes it works well

 

 
{code:java}
import java.lang.{Double => JDouble}

val sumAgg = new Aggregator[Double, JDouble, JDouble] {
  def zero: JDouble = null

  def reduce(b: JDouble, a: Double): JDouble =
    if (b == null) {
      a
    } else {
      b + a
    }

  def merge(b1: JDouble, b2: JDouble): JDouble =
    if (b1 == null) {
      b2
    } else if (b2 == null) {
      b1
    } else {
      b1 + b2
    }

  def finish(r: JDouble): JDouble = r

  def bufferEncoder: Encoder[JDouble] = ExpressionEncoder()
  def outputEncoder: Encoder[JDouble] = ExpressionEncoder()
}

val df = Seq.empty[Double]
  .toDF()
  .select(udaf(sumAgg).apply(col("value")))

df.printSchema()
root
 |-- $anon$3(value): double (nullable = true) 

df.show()
+--------------+
|$anon$3(value)|
+--------------+
|          null|
+--------------+
 {code}
it works with Option without issues too

 

> udaf doesnt work with nullable (or option of) 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
>            Priority: Minor
>
> i have a need to have a dataframe 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}
> case class SumAndProduct(sum: Double, product: Double)
> 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}
> i dont really understand the error, this result is not a top-level row object.
> anyhow taking the advice to heart and using option we get to the second attempt using options:
> {code:java}
> case class SumAndProduct(sum: Double, product: Double) 
> 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