zhidongqu-db commented on code in PR #54011:
URL: https://github.com/apache/spark/pull/54011#discussion_r2743250956
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/vectorExpressions.scala:
##########
@@ -328,3 +344,469 @@ case class VectorNormalize(vector: Expression, degree:
Expression)
copy(vector = newChildren(0), degree = newChildren(1))
}
}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(array) - Returns the element-wise mean of float vectors in a group.
+ All vectors must have the same dimension.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(col) FROM VALUES (array(1.0F, 2.0F)), (array(3.0F,
4.0F)) AS tab(col);
+ [2.0,3.0]
+ """,
+ since = "4.2.0",
+ group = "vector_funcs"
+)
+// scalastyle:on line.size.limit
+case class VectorAvg(
+ child: Expression,
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0
+) extends ImperativeAggregate
+ with UnaryLike[Expression]
+ with QueryErrorsBase {
+
+ def this(child: Expression) = this(child, 0, 0)
+
+ override def prettyName: String = "vector_avg"
+
+ override def nullable: Boolean = true
+
+ override def dataType: DataType = ArrayType(FloatType, containsNull = false)
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ child.dataType match {
+ case ArrayType(FloatType, _) =>
+ TypeCheckResult.TypeCheckSuccess
+ case _ =>
+ DataTypeMismatch(
+ errorSubClass = "UNEXPECTED_INPUT_TYPE",
+ messageParameters = Map(
+ "paramIndex" -> ordinalNumber(0),
+ "requiredType" -> toSQLType(ArrayType(FloatType)),
+ "inputSql" -> toSQLExpr(child),
+ "inputType" -> toSQLType(child.dataType)
+ )
+ )
+ }
+ }
+
+ // Aggregate buffer schema: (avg: BINARY, dim: INTEGER, count: LONG)
+ // avg is a BINARY representation of the average vector of floats in the
group
+ // dim is the dimension of the vector
+ // count is the number of vectors in the group
+ // null avg means no valid input has been seen yet
+ private lazy val avgAttr = AttributeReference(
+ "avg",
+ BinaryType,
+ nullable = true
+ )()
+ private lazy val dimAttr = AttributeReference(
+ "dim",
+ IntegerType,
+ nullable = true
+ )()
+ private lazy val countAttr =
+ AttributeReference("count", LongType, nullable = false)()
+
+ override def aggBufferSchema: StructType = StructType(
Review Comment:
done
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/vectorExpressions.scala:
##########
@@ -328,3 +344,469 @@ case class VectorNormalize(vector: Expression, degree:
Expression)
copy(vector = newChildren(0), degree = newChildren(1))
}
}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(array) - Returns the element-wise mean of float vectors in a group.
+ All vectors must have the same dimension.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(col) FROM VALUES (array(1.0F, 2.0F)), (array(3.0F,
4.0F)) AS tab(col);
+ [2.0,3.0]
+ """,
+ since = "4.2.0",
+ group = "vector_funcs"
+)
+// scalastyle:on line.size.limit
+case class VectorAvg(
+ child: Expression,
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0
+) extends ImperativeAggregate
+ with UnaryLike[Expression]
+ with QueryErrorsBase {
+
+ def this(child: Expression) = this(child, 0, 0)
+
+ override def prettyName: String = "vector_avg"
+
+ override def nullable: Boolean = true
+
+ override def dataType: DataType = ArrayType(FloatType, containsNull = false)
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ child.dataType match {
+ case ArrayType(FloatType, _) =>
+ TypeCheckResult.TypeCheckSuccess
+ case _ =>
+ DataTypeMismatch(
+ errorSubClass = "UNEXPECTED_INPUT_TYPE",
+ messageParameters = Map(
+ "paramIndex" -> ordinalNumber(0),
+ "requiredType" -> toSQLType(ArrayType(FloatType)),
+ "inputSql" -> toSQLExpr(child),
+ "inputType" -> toSQLType(child.dataType)
+ )
+ )
+ }
+ }
+
+ // Aggregate buffer schema: (avg: BINARY, dim: INTEGER, count: LONG)
+ // avg is a BINARY representation of the average vector of floats in the
group
+ // dim is the dimension of the vector
+ // count is the number of vectors in the group
+ // null avg means no valid input has been seen yet
+ private lazy val avgAttr = AttributeReference(
+ "avg",
+ BinaryType,
+ nullable = true
+ )()
+ private lazy val dimAttr = AttributeReference(
+ "dim",
+ IntegerType,
+ nullable = true
+ )()
+ private lazy val countAttr =
+ AttributeReference("count", LongType, nullable = false)()
+
+ override def aggBufferSchema: StructType = StructType(
+ Seq(
+ StructField(
+ "avg",
+ BinaryType,
+ nullable = true
+ ),
+ StructField("dim", IntegerType, nullable = true),
+ StructField("count", LongType, nullable = false)
+ )
+ )
+
+ override def aggBufferAttributes: Seq[AttributeReference] =
+ Seq(avgAttr, dimAttr, countAttr)
+
+ override lazy val inputAggBufferAttributes: Seq[AttributeReference] =
+ aggBufferAttributes.map(_.newInstance())
+
+ // Buffer indices
+ private val avgIndex = 0
+ private val dimIndex = 1
+ private val countIndex = 2
+
+ override def withNewMutableAggBufferOffset(
+ newMutableAggBufferOffset: Int
+ ): ImperativeAggregate =
+ copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+ override def withNewInputAggBufferOffset(
+ newInputAggBufferOffset: Int
+ ): ImperativeAggregate =
+ copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+ override def initialize(buffer: InternalRow): Unit = {
+ buffer.update(mutableAggBufferOffset + avgIndex, null)
+ buffer.update(mutableAggBufferOffset + dimIndex, null)
+ buffer.setLong(mutableAggBufferOffset + countIndex, 0L)
+ }
+
+ override def update(buffer: InternalRow, input: InternalRow): Unit = {
+ val inputValue = child.eval(input)
+ if (inputValue == null) {
+ return
+ }
+
+ val inputArray = inputValue.asInstanceOf[ArrayData]
+ val inputLen = inputArray.numElements()
+
+ // Check for NULL elements in input vector - skip if any NULL element found
+ for (i <- 0 until inputLen) {
+ if (inputArray.isNullAt(i)) {
+ return
+ }
+ }
+
+ val currentCount = buffer.getLong(mutableAggBufferOffset + countIndex)
+
+ if (currentCount == 0L) {
+ // First valid vector - just copy it as the initial average
+ val byteBuffer =
+ ByteBuffer.allocate(inputLen * 4).order(ByteOrder.LITTLE_ENDIAN)
+ for (i <- 0 until inputLen) {
+ byteBuffer.putFloat(inputArray.getFloat(i))
+ }
+ buffer.update(mutableAggBufferOffset + avgIndex, byteBuffer.array())
+ buffer.setInt(mutableAggBufferOffset + dimIndex, inputLen)
+ buffer.setLong(mutableAggBufferOffset + countIndex, 1L)
+ } else {
+ val currentDim = buffer.getInt(mutableAggBufferOffset + dimIndex)
+
+ // Empty array case - if current is empty and input is empty, keep empty
+ if (currentDim == 0 && inputLen == 0) {
+ buffer.setLong(mutableAggBufferOffset + countIndex, currentCount + 1L)
+ return
+ }
+
+ // Dimension mismatch check
+ if (currentDim != inputLen) {
+ throw QueryExecutionErrors.vectorDimensionMismatchError(
+ prettyName,
+ currentDim,
+ inputLen
+ )
+ }
+
+ // Update running average: new_avg = old_avg + (new_value - old_avg) /
(count + 1)
+ val newCount = currentCount + 1L
+ val currentAvgBytes = buffer.getBinary(mutableAggBufferOffset + avgIndex)
+ val currentAvgBuffer =
+ ByteBuffer.wrap(currentAvgBytes).order(ByteOrder.LITTLE_ENDIAN)
+ val newAvgBuffer =
+ ByteBuffer.allocate(currentDim * 4).order(ByteOrder.LITTLE_ENDIAN)
+ for (i <- 0 until currentDim) {
+ val oldAvg = currentAvgBuffer.getFloat()
+ val newVal = inputArray.getFloat(i)
+ newAvgBuffer.putFloat(oldAvg + ((newVal - oldAvg) / newCount.toFloat))
+ }
+ buffer.update(mutableAggBufferOffset + avgIndex, newAvgBuffer.array())
+ buffer.setLong(mutableAggBufferOffset + countIndex, newCount)
+ }
+ }
+
+ override def merge(buffer: InternalRow, inputBuffer: InternalRow): Unit = {
+ val inputCount = inputBuffer.getLong(inputAggBufferOffset + countIndex)
+ if (inputCount == 0L) {
+ return
+ }
+
+ val inputAvgBytes = inputBuffer.getBinary(inputAggBufferOffset + avgIndex)
+ val inputDim = inputBuffer.getInt(inputAggBufferOffset + dimIndex)
+ val currentCount = buffer.getLong(mutableAggBufferOffset + countIndex)
+
+ if (currentCount == 0L) {
+ // Copy input buffer to current buffer
+ buffer.update(mutableAggBufferOffset + avgIndex, inputAvgBytes.clone())
+ buffer.setInt(mutableAggBufferOffset + dimIndex, inputDim)
+ buffer.setLong(mutableAggBufferOffset + countIndex, inputCount)
+ } else {
+ val currentDim = buffer.getInt(mutableAggBufferOffset + dimIndex)
+
+ // Empty array case
+ if (currentDim == 0 && inputDim == 0) {
+ buffer.setLong(
+ mutableAggBufferOffset + countIndex,
+ currentCount + inputCount
+ )
+ return
+ }
+
+ // Dimension mismatch check
+ if (currentDim != inputDim) {
+ throw QueryExecutionErrors.vectorDimensionMismatchError(
+ prettyName,
+ currentDim,
+ inputDim
+ )
+ }
+
+ // Merge running averages:
+ // combined_avg = (left_avg * left_count) / (left_count + right_count) +
+ // (right_avg * right_count) / (left_count + right_count)
+ val newCount = currentCount + inputCount
+ val currentAvgBytes = buffer.getBinary(mutableAggBufferOffset + avgIndex)
+ val currentAvgBuffer =
+ ByteBuffer.wrap(currentAvgBytes).order(ByteOrder.LITTLE_ENDIAN)
+ val inputAvgBuffer =
+ ByteBuffer.wrap(inputAvgBytes).order(ByteOrder.LITTLE_ENDIAN)
+ val newAvgBuffer =
+ ByteBuffer.allocate(currentDim * 4).order(ByteOrder.LITTLE_ENDIAN)
+ for (_ <- 0 until currentDim) {
+ // getFloat() will auto-increment the buffer's current position by 4
+ val leftAvg = currentAvgBuffer.getFloat()
+ val rightAvg = inputAvgBuffer.getFloat()
+ newAvgBuffer.putFloat(
+ (leftAvg * currentCount) / newCount.toFloat +
+ (rightAvg * inputCount) / newCount.toFloat
+ )
+ }
+ buffer.update(mutableAggBufferOffset + avgIndex, newAvgBuffer.array())
+ buffer.setLong(mutableAggBufferOffset + countIndex, newCount)
+ }
+ }
+
+ override def eval(buffer: InternalRow): Any = {
+ val count = buffer.getLong(mutableAggBufferOffset + countIndex)
+ if (count == 0L) {
+ null
+ } else {
+ val dim = buffer.getInt(mutableAggBufferOffset + dimIndex)
+ val avgBytes = buffer.getBinary(mutableAggBufferOffset + avgIndex)
+ val avgBuffer = ByteBuffer.wrap(avgBytes).order(ByteOrder.LITTLE_ENDIAN)
+ val result = new Array[Float](dim)
+ for (i <- 0 until dim) {
+ result(i) = avgBuffer.getFloat()
+ }
+ ArrayData.toArrayData(result)
+ }
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): VectorAvg
=
+ copy(child = newChild)
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(array) - Returns the element-wise sum of float vectors in a group.
+ All vectors must have the same dimension.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(col) FROM VALUES (array(1.0F, 2.0F)), (array(3.0F,
4.0F)) AS tab(col);
+ [4.0,6.0]
+ """,
+ since = "4.2.0",
+ group = "vector_funcs"
+)
+// scalastyle:on line.size.limit
+case class VectorSum(
+ child: Expression,
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0
+) extends ImperativeAggregate
+ with UnaryLike[Expression]
+ with QueryErrorsBase {
+
+ def this(child: Expression) = this(child, 0, 0)
+
+ override def prettyName: String = "vector_sum"
+
+ override def nullable: Boolean = true
+
+ override def dataType: DataType = ArrayType(FloatType, containsNull = false)
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ child.dataType match {
+ case ArrayType(FloatType, _) =>
+ TypeCheckResult.TypeCheckSuccess
+ case _ =>
+ DataTypeMismatch(
+ errorSubClass = "UNEXPECTED_INPUT_TYPE",
+ messageParameters = Map(
+ "paramIndex" -> ordinalNumber(0),
+ "requiredType" -> toSQLType(ArrayType(FloatType)),
+ "inputSql" -> toSQLExpr(child),
+ "inputType" -> toSQLType(child.dataType)
+ )
+ )
+ }
+ }
+
+ // Aggregate buffer schema: (sum: BINARY, dim: INTEGER)
+ // sum is a BINARY representation of the sum vector of floats in the group
+ // dim is the dimension of the vector
+ // null sum means no valid input has been seen yet
+ private lazy val sumAttr = AttributeReference(
+ "sum",
+ BinaryType,
+ nullable = true
+ )()
+ private lazy val dimAttr = AttributeReference(
+ "dim",
+ IntegerType,
+ nullable = true
+ )()
+
+ override def aggBufferSchema: StructType = StructType(
Review Comment:
done
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]