cloud-fan commented on a change in pull request #25024: [SPARK-27296][SQL] 
Allows Aggregator to be registered as a UDF
URL: https://github.com/apache/spark/pull/25024#discussion_r363154100
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
 ##########
 @@ -450,3 +454,88 @@ case class ScalaUDAF(
 
   override def nodeName: String = udaf.getClass.getSimpleName
 }
+
+case class ScalaAggregator[IN, BUF, OUT](
+    children: Seq[Expression],
+    agg: Aggregator[IN, BUF, OUT],
+    inputEncoderNR: ExpressionEncoder[IN],
+    isNullable: Boolean = true,
+    isDeterministic: Boolean = true,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[BUF]
+  with NonSQLExpression
+  with UserDefinedExpression
+  with ImplicitCastInputTypes
+  with Logging {
+
+  private[this] lazy val inputEncoder = inputEncoderNR.resolveAndBind()
+  private[this] lazy val bufferEncoder = 
agg.bufferEncoder.asInstanceOf[ExpressionEncoder[BUF]]
+  private[this] lazy val outputEncoder = 
agg.outputEncoder.asInstanceOf[ExpressionEncoder[OUT]]
+
+  def dataType: DataType = outputEncoder.objSerializer.dataType
+
+  def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType)
+
+  def nullable: Boolean = isNullable
+
+  override lazy val deterministic: Boolean = isDeterministic
+
+  def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): 
ScalaAggregator[IN, BUF, OUT] =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): 
ScalaAggregator[IN, BUF, OUT] =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  private[this] lazy val childrenSchema: StructType = {
+    val inputFields = children.zipWithIndex.map {
+      case (child, index) =>
+        StructField(s"input$index", child.dataType, child.nullable, 
Metadata.empty)
+    }
+    StructType(inputFields)
+  }
+
+  private[this] lazy val inputProjection = {
+    val inputAttributes = childrenSchema.toAttributes
+    log.debug(
+      s"Creating MutableProj: $children, inputSchema: $inputAttributes.")
+    UnsafeProjection.create(children, inputAttributes)
+  }
+
+  def createAggregationBuffer(): BUF = agg.zero
+
+  def update(buffer: BUF, input: InternalRow): BUF = {
+    val proj = inputProjection(input)
+    val a = inputEncoder.fromRow(proj)
+    agg.reduce(buffer, a)
+  }
+
+  def merge(buffer: BUF, input: BUF): BUF = agg.merge(buffer, input)
+
+  def eval(buffer: BUF): Any = {
+    val row = outputEncoder.toRow(agg.finish(buffer))
+    if (outputEncoder.isSerializedAsStruct) row else row.get(0, dataType)
+  }
+
+  private[this] lazy val bufferRow = {
+    val bufferSerializer = bufferEncoder.namedExpressions
+    new UnsafeRow(bufferSerializer.length)
+  }
+  private[this] lazy val bufferRowToObject = {
+    val bufferDeserializer = bufferEncoder.resolveAndBind().deserializer
+    GenerateSafeProjection.generate(bufferDeserializer :: Nil)
+  }
+
+  def serialize(agg: BUF): Array[Byte] =
+    bufferEncoder.toRow(agg).asInstanceOf[UnsafeRow].getBytes()
+
+  def deserialize(storageFormat: Array[Byte]): BUF = {
+    bufferRow.pointTo(storageFormat, storageFormat.length)
+    bufferRowToObject(bufferRow).get(0, 
ObjectType(classOf[Any])).asInstanceOf[BUF]
 
 Review comment:
   we can call `bufferEncoder.fromRow` here, but we need to resolve and bind 
`bufferEncoder` first
   ```
   lazy val bufferEncoder = 
agg.bufferEncoder.asInstanceOf[ExpressionEncoder[BUF]].resolveAndBind()
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to