You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/02/25 15:21:06 UTC

[GitHub] cloud-fan commented on a change in pull request #23778: [SPARK-24935][SQL] : Problem with Executing Hive UDF's from Spark 2.2 Onwards

cloud-fan commented on a change in pull request #23778: [SPARK-24935][SQL] : Problem with Executing Hive UDF's from Spark 2.2 Onwards
URL: https://github.com/apache/spark/pull/23778#discussion_r259873144
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
 ##########
 @@ -524,23 +524,142 @@ abstract class TypedImperativeAggregate[T] extends ImperativeAggregate {
   /** De-serializes the serialized format Array[Byte], and produces aggregation buffer object T */
   def deserialize(storageFormat: Array[Byte]): T
 
+  override def initialize(buffer: InternalRow): Unit = {
+    buffer(mutableAggBufferOffset) = createAggregationBuffer()
+  }
+
+  override def update(buffer: InternalRow, input: InternalRow): Unit = {
+    buffer(mutableAggBufferOffset) = update(getBufferObject(buffer), input)
+  }
+
+  override def merge(buffer: InternalRow, inputBuffer: InternalRow): Unit = {
+    val bufferObject = getBufferObject(buffer)
+    // The inputBuffer stores serialized aggregation buffer object produced by partial aggregate
+    val inputObject = deserialize(inputBuffer.getBinary(inputAggBufferOffset))
+    buffer(mutableAggBufferOffset) = merge(bufferObject, inputObject)
+  }
+
+  override def eval(buffer: InternalRow): Any = {
+    eval(getBufferObject(buffer))
+  }
+
+  private[this] val anyObjectType = ObjectType(classOf[AnyRef])
+  private def getBufferObject(bufferRow: InternalRow): T = {
+    bufferRow.get(mutableAggBufferOffset, anyObjectType).asInstanceOf[T]
+  }
+
+  override lazy val aggBufferAttributes: Seq[AttributeReference] = {
+    // Underlying storage type for the aggregation buffer object
+    Seq(AttributeReference("buf", BinaryType)())
+  }
+
+  override lazy val inputAggBufferAttributes: Seq[AttributeReference] =
+    aggBufferAttributes.map(_.newInstance())
+
+  override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes)
+
+  /**
+   * In-place replaces the aggregation buffer object stored at buffer's index
+   * `mutableAggBufferOffset`, with SparkSQL internally supported underlying storage format
+   * (BinaryType).
+   *
+   * This is only called when doing Partial or PartialMerge mode aggregation, before the framework
+   * shuffle out aggregate buffers.
+   */
+  def serializeAggregateBufferInPlace(buffer: InternalRow): Unit = {
+    buffer(mutableAggBufferOffset) = serialize(getBufferObject(buffer))
+  }
+}
+
+/**
+ * Aggregation function which allows **arbitrary** user-defined java object to be used as internal
+ * aggregation buffer for Hive.
+ */
+abstract class HiveTypedImperativeAggregate[T] extends TypedImperativeAggregate[T] {
+
+  /**
+   * Creates an empty aggregation buffer object for partial 1 mode. This is called
+   * before processing each key group(group by key).
+   *
+   * @return an aggregation buffer object
+   */
+  def createAggregationBuffer(): T
+
+  /**
+   * Creates an empty aggregation buffer object for partial 2 mode.
+   *
+   * @return an aggregation buffer object
+   */
+  def createPartial2ModeAggregationBuffer(): T
+
+  var partial2ModeBuffer: InternalRow = _
+
+  /**
+   * Updates the aggregation buffer object with an input row and returns a new buffer object. For
+   * performance, the function may do in-place update and return it instead of constructing new
+   * buffer object.
+   *
+   * This is typically called when doing Partial or Complete mode aggregation.
+   *
+   * @param buffer The aggregation buffer object.
+   * @param input an input row
+   */
+  def update(buffer: T, input: InternalRow): T
+
+  /**
+   * Merges an input aggregation object into aggregation buffer object and returns a new buffer
+   * object. For performance, the function may do in-place merge and return it instead of
+   * constructing new buffer object.
+   *
+   * This is typically called when doing PartialMerge or Final mode aggregation.
+   *
+   * @param buffer the aggregation buffer object used to store the aggregation result.
+   * @param input an input aggregation object. Input aggregation object can be produced by
+   *              de-serializing the partial aggregate's output from Mapper side.
+   */
+  def merge(buffer: T, input: T): T
+
+  /**
+   * Generates the final aggregation result value for current key group with the aggregation buffer
+   * object.
+   *
+   * Developer note: the only return types accepted by Spark are:
+   *   - primitive types
+   *   - InternalRow and subclasses
+   *   - ArrayData
+   *   - MapData
+   *
+   * @param buffer aggregation buffer object.
+   * @return The aggregation result of current key group
+   */
+  def eval(buffer: T): Any
+
+  /** Serializes the aggregation buffer object T to Array[Byte] */
+  def serialize(buffer: T): Array[Byte]
+
+  /** De-serializes the serialized format Array[Byte], and produces aggregation buffer object T */
+  def deserialize(storageFormat: Array[Byte]): T
+
   final override def initialize(buffer: InternalRow): Unit = {
+    partial2ModeBuffer = buffer.copy()
+    partial2ModeBuffer(mutableAggBufferOffset) = createPartial2ModeAggregationBuffer()
 
 Review comment:
   I'm a little lost here. So this `HiveTypedImperativeAggregate` has 2 buffers? What's the difference between `partial2ModeBuffer` and `buffer`?

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


With regards,
Apache Git Services

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