mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1164405383


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,523 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.util.Locale
+
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.WritableMemory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import 
org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, 
TypeCheckSuccess}
+import org.apache.spark.sql.catalyst.expressions.{Expression, 
ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.catalyst.util.TypeUtils.{toSQLExpr, toSQLId, 
toSQLType, toSQLValue}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, 
LongType, NullType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * This datasketchesAggregates file is intended to encapsulate all of the
+ * aggregate functions that utilize Datasketches sketch objects as intermediate
+ * aggregation buffers.
+ *
+ * The HllSketchAggregate sealed trait is meant to be extended by the aggregate
+ * functions which utilize instances of HllSketch to count uniques.
+ */
+sealed trait HllSketchAggregate
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree 
transformation.
+
+  lazy val lgConfigK: Int = second.eval().asInstanceOf[Int]
+  lazy val tgtHllType: TgtHllType =
+    
TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+
+  // Type checking
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        DataTypeMismatch(
+          errorSubClass = "UNEXPECTED_NULL",
+          messageParameters = Map(
+            "exprName" -> "lgConfigK or tgtHllType"
+          )
+        )
+      case (_, IntegerType, StringType) =>
+        if (!second.foldable) {
+          DataTypeMismatch(
+            errorSubClass = "NON_FOLDABLE_INPUT",
+            messageParameters = Map(
+              "inputName" -> "lgConfigK",
+              "inputType" -> toSQLType(second.dataType),
+              "inputExpr" -> toSQLExpr(second)
+            )
+          )
+        } else if (lgConfigK <= 0L) {
+          DataTypeMismatch(
+            errorSubClass = "VALUE_OUT_OF_RANGE",
+            messageParameters = Map(
+              "exprName" -> "lgConfigK",
+              "valueRange" -> s"[0, positive]",
+              "currentValue" -> toSQLValue(lgConfigK, IntegerType)
+            )
+          )
+        } else if (!third.foldable) {
+          DataTypeMismatch(
+            errorSubClass = "NON_FOLDABLE_INPUT",
+            messageParameters = Map(
+              "inputName" -> "numBitsExpression",
+              "inputType" -> toSQLType(third.dataType),
+              "inputExpr" -> toSQLExpr(third)
+            )
+          )
+        } else {
+          TypeCheckSuccess
+        }
+      case _ =>
+        DataTypeMismatch(
+          errorSubClass = "HLLSKETCH_WRONG_TYPE",
+          messageParameters = Map(
+            "functionName" -> toSQLId(prettyName),
+            "expectedSecond" -> toSQLType(IntegerType),
+            "expectedThird" -> toSQLType(StringType),
+            "actual" -> Seq(first.dataType, second.dataType, third.dataType)
+              .map(toSQLType).mkString(", ")
+          )
+        )
+    }
+  }
+
+  // From here on, these are the shared default implementations for 
TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never 
return null */
+  override def nullable: Boolean = false
+
+  /**
+   * Instantiate an HllSketch instance using the lgConfigK and tgtHllType 
params.
+   *
+   * @return an HllSketch instance
+   */
+  override def createAggregationBuffer(): HllSketch = {
+    new HllSketch(lgConfigK, tgtHllType)
+  }
+
+  /**
+   * Evaluate the input row and update the HllSketch instance with the row's 
value.
+   * The update function only supports a subset of Spark SQL types, and an
+   * UnsupportedOperationException will be thrown for unsupported types.
+   *
+   * @param sketch The HllSketch instance.
+   * @param input  an input row
+   */
+  override def update(sketch: HllSketch, input: InternalRow): HllSketch = {
+    val v = first.eval(input)
+    if (v != null) {
+      first.dataType match {
+        // Update implemented for a subset of types supported by HllSketch
+        // Spark SQL doesn't have equivalent types for ByteBuffer or char[] so 
leave those out
+        // Leaving out support for Array types, as unique counting these 
aren't a common use case
+        // Leaving out support for floating point types (IE DoubleType) due to 
imprecision
+        // TODO: implement support for decimal/datetime/interval types
+        case IntegerType => sketch.update(v.asInstanceOf[Int])
+        case LongType => sketch.update(v.asInstanceOf[Long])
+        case StringType => sketch.update(v.asInstanceOf[UTF8String].toString)
+        case dataType => throw new UnsupportedOperationException(
+          s"A HllSketch instance cannot be updates with a Spark 
${dataType.toString} type")
+      }
+    }
+    sketch
+  }
+
+  /**
+   * Merges an input HllSketch into the sketch which is acting as the 
aggregation buffer.
+   *
+   * @param sketch the HllSketch instance used to store the aggregation result.
+   * @param input an input HllSketch instance
+   */
+  override def merge(sketch: HllSketch, input: HllSketch): HllSketch = {
+      val union = new Union(sketch.getLgConfigK)
+      union.update(sketch)
+      union.update(input)
+      union.getResult(sketch.getTgtHllType)
+  }
+
+  /** Convert the underlying HllSketch into an updateable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toCompactByteArray
+  }

Review Comment:
   I prefer that we serialize to an updatable byte array. I believe 
deserializing from that is faster. Thank you for looking into that.



-- 
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]

Reply via email to