You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "mkaravel (via GitHub)" <gi...@apache.org> on 2023/04/10 16:46:31 UTC

[GitHub] [spark] mkaravel commented on a diff in pull request #40615: [WIP][SPARK-16484][SQL] Add support for Datasketches HllSketch

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


##########
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala:
##########
@@ -1545,6 +1545,158 @@ class DataFrameAggregateSuite extends QueryTest
     )
     checkAnswer(res, Row(Array(1), Array(1)))
   }
+
+  test("SPARK-16484: hllsketch_estimate positive tests") {
+    val df1 = Seq(
+      (1, "a"), (1, "a"), (1, "a"),
+      (1, "b"),
+      (1, "c"), (1, "c"),
+      (1, "d")
+    ).toDF("id", "value")
+    df1.createOrReplaceTempView("df1")
+
+    val df2 = Seq(
+      (1, "a"),
+      (1, "c"),
+      (1, "d"), (1, "d"), (1, "d"),
+      (1, "e"), (1, "e"),
+      (1, "f")
+    ).toDF("id", "value")
+    df2.createOrReplaceTempView("df2")
+
+    // empty column test
+    val res0 = df1.withColumn("empty_col", lit(null)).groupBy("id")
+      .agg(
+        hllsketch_estimate("empty_col").as("distinct_count")
+      )
+    checkAnswer(res0, Row(1, 0))
+
+    // first test hllsketch_estimate via dataframe + sql, with and without configs
+    val res1 = df1.groupBy("id")
+      .agg(
+        count("value").as("count"),
+        hllsketch_estimate("value").as("distinct_count_1"),
+        hllsketch_estimate("value", 20, "HLL_8").as("distinct_count_2")
+      )
+    checkAnswer(res1, Row(1, 7, 4, 4))
+
+    val res2 = sql(
+      """select
+        | id,
+        | count(value) as count,
+        | hllsketch_estimate(value) as distinct_count_1,
+        | hllsketch_estimate(value, 20, 'HLL_8') as distinct_count_2
+        |from df1
+        |group by 1
+        |""".stripMargin)
+    checkAnswer(res2, Row(1, 7, 4, 4))
+
+    // next test hllsketch_binary via dataframe + sql, with and without configs
+    val df3 = df1.groupBy("id")
+      .agg(
+        count("value").as("count"),
+        hllsketch_binary("value").as("hllsketch_1"),
+        hllsketch_binary("value", 20, "HLL_8").as("hllsketch_2"),
+        hllsketch_binary("value").as("hllsketch_3")
+      )
+    df3.createOrReplaceTempView("df3")
+
+    // now test hllsketch_union_estimate via dataframe + sql, with and without configs,
+    // unioning together sketches with default, non-default and different configurations
+    val df4 = sql(
+      """select
+        | id,
+        | count(value),
+        | hllsketch_binary(value) as hllsketch_1,
+        | hllsketch_binary(value, 20, 'HLL_8') as hllsketch_2,
+        | hllsketch_binary(value, 20, 'HLL_8') as hllsketch_3
+        |from df2
+        |group by 1
+        |""".stripMargin)
+    df4.createOrReplaceTempView("df4")
+
+    val res3 = df3.union(df4).groupBy("id")
+      .agg(
+        sum("count").as("count"),
+        hllsketch_union_estimate("hllsketch_1", 20).as("distinct_count_1"),
+        hllsketch_union_estimate("hllsketch_2").as("distinct_count_2"),
+        hllsketch_union_estimate("hllsketch_3").as("distinct_count_3")
+      )
+    checkAnswer(res3, Row(1, 15, 6, 6, 6))
+
+    val res4 = sql(
+      """select
+        | id,
+        | sum(count) as count,
+        | hllsketch_union_estimate(hllsketch_1, 20) as distinct_count_1,
+        | hllsketch_union_estimate(hllsketch_2) as distinct_count_2,
+        | hllsketch_union_estimate(hllsketch_3) as distinct_count_3
+        |from (select * from df3 union all select * from df4)
+        |group by 1
+        |""".stripMargin)
+    checkAnswer(res4, Row(1, 15, 6, 6, 6))
+  }
+
+  test("SPARK-16484: hllsketch_estimate negative tests") {
+
+    val df1 = Seq(
+      (1, "a"), (1, "a"), (1, "a"),
+      (1, "b"),
+      (1, "c"), (1, "c"),
+      (1, "d")
+    ).toDF("id", "value")
+
+    // validate that the functions error out when lgConfigK < 0
+    val error0 = intercept[AnalysisException] {
+      val res = df1.groupBy("id")
+        .agg(
+          hllsketch_estimate("value", -1, "HLL_4").as("hllsketch")
+        )
+      checkAnswer(res, Nil)
+    }
+    assert(error0.toString contains "DATATYPE_MISMATCH")

Review Comment:
   This does not seem like a good error message. It's a bad/invalid lgk value not a wrong data type.
   
   Also I would add tests for values just below and above the acceptable range. I believe the acceptable range is between 4 and 21 (inclusive), so I would expect tests with `3` and `22`.



##########
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala:
##########
@@ -1545,6 +1545,59 @@ class DataFrameAggregateSuite extends QueryTest
     )
     checkAnswer(res, Row(Array(1), Array(1)))
   }
+
+  test("SPARK-16484: hllsketch_estimate positive tests") {

Review Comment:
   Some more testing ideas:
   * Table that contains only `NULL` values.
   * Table that contains `NULL` and non-`NULL` values.
   * Table with two columns where the first column is used for grouping (grouping aggregation) and the second is the one we want to compute the sketch for. In that table, I would like to see one group that has only `NULL` values, and one group that has both `NULL` and non-`NULL` values. So we are talking about at least 2 different groups.



##########
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
+  }
+
+  /** De-serializes the updateable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllSketchEstimate function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgConfigK the log-base-2 of K, where K is the number of buckets or slots for the sketch
+ * @param tgtHllType the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8)
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgConfigK, tgtHllType]) - Returns the estimated number of unique values.
+      `lgConfigK` the log-base-2 of K, where K is the number of buckets or slots for the HllSketch.
+      `tgtHllType` the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends HllSketchAggregate {
+
+  // all the constructors, case class's apply doesn't work for sql
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression, tgtHllType: Expression) = {
+    this(child, lgConfigK, tgtHllType, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int, tgtHllType: String) = {
+    this(child, Literal(lgConfigK), Literal(tgtHllType), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchEstimate =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchEstimate =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchEstimate =
+    copy(child = newFirst, lgConfigKExpression = newSecond, tgtHllTypeExpression = newThird)
+
+  // overrides for TernaryLike
+
+  override def first: Expression = child
+  override def second: Expression = lgConfigKExpression
+  override def third: Expression = tgtHllTypeExpression
+
+  // Implementations specific to HllSketchEstimate
+
+  override def prettyName: String = "hllsketch_estimate"
+
+  override def dataType: DataType = LongType
+
+  /**
+   * Returns the estimated number of unique values
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A long value representing the number of uniques
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.getEstimate.toLong
+  }
+}
+
+/**
+ * The HllSketchBinary function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the compact binary representation of the HllSketch.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgConfigK the log-base-2 of K, where K is the number of buckets or slots for the sketch
+ * @param tgtHllType the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8)
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgConfigK, tgtHllType]) - Returns the HllSketch's compact binary representation.
+      `lgConfigK` the log-base-2 of K, where K is the number of buckets or slots for the HllSketch.
+      `tgtHllType` the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hllsketch_binary_estimate(_FUNC_(col1))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchBinary(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends HllSketchAggregate {
+
+  // all the constructors, case class's apply doesn't work for sql
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression, tgtHllType: Expression) = {
+    this(child, lgConfigK, tgtHllType, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int, tgtHllType: String) = {
+    this(child, Literal(lgConfigK), Literal(tgtHllType), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchBinary =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchBinary =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchBinary =
+    copy(child = newFirst, lgConfigKExpression = newSecond, tgtHllTypeExpression = newThird)
+
+  // overrides for TernaryLike
+
+  override def first: Expression = child
+  override def second: Expression = lgConfigKExpression
+  override def third: Expression = tgtHllTypeExpression
+
+  // Implementations specific to HllSketchBinary
+
+  override def prettyName: String = "hllsketch_binary"
+
+  override def dataType: DataType = BinaryType
+
+  /**
+   * Returns the compact byte array associated with the HllSketch
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary value which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+}
+
+/**
+ * The HllSketchUnionEstimate function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the estimated unique count from the merged HllSketches.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgMaxK The largest maximum size for lgConfigK for the union operation.
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgMaxK]) - Returns the estimated number of unique values.
+      `lgMaxK` The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hllsketch_binary(col1))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchUnionEstimate(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // all the constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgMaxK: Expression) = {
+    this(child, lgMaxK, 0, 0)
+  }
+
+  def this(child: Expression, lgMaxK: Int) = {
+    this(child, Literal(lgMaxK), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllSketchUnionEstimate = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchUnionEstimate =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllSketchUnionEstimate = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // overrides for BinaryLike
+
+  override def left: Expression = child
+  override def right: Expression = lgMaxKExpression
+
+  // Type checking
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (left.dataType, right.dataType) match {
+      case (_, NullType) =>
+        DataTypeMismatch(
+          errorSubClass = "UNEXPECTED_NULL",
+          messageParameters = Map(
+            "exprName" -> "lgMaxK"
+          )
+        )
+      case (BinaryType, IntegerType) =>
+        if (!right.foldable) {
+          DataTypeMismatch(
+            errorSubClass = "NON_FOLDABLE_INPUT",
+            messageParameters = Map(
+              "inputName" -> "lgMaxK",
+              "inputType" -> toSQLType(right.dataType),
+              "inputExpr" -> toSQLExpr(right)
+            )
+          )
+        } else if (lgMaxK <= 0L) {
+          DataTypeMismatch(
+            errorSubClass = "VALUE_OUT_OF_RANGE",
+            messageParameters = Map(
+              "exprName" -> "lgMaxK",
+              "valueRange" -> s"[0, positive]",
+              "currentValue" -> toSQLValue(lgMaxK, IntegerType)
+            )
+          )
+        } else {
+          TypeCheckSuccess
+        }
+      case _ =>
+        DataTypeMismatch(
+          errorSubClass = "HLLSKETCH_WRONG_TYPE",
+          messageParameters = Map(
+            "functionName" -> toSQLId(prettyName),
+            "expectedRight" -> toSQLType(IntegerType),
+            "actual" -> Seq(left.dataType, right.dataType)
+              .map(toSQLType).mkString(", ")
+          )
+        )
+    }
+  }
+
+  // Implementations specific to HllSketchUnionEstimate
+
+  override def prettyName: String = "hllsketch_union_estimate"
+
+  override def dataType: DataType = LongType
+
+  override def nullable: Boolean = false
+
+  /**
+   * Instantiate an Union instance using the lgMaxK param.
+   *
+   * @return an Union instance
+   */
+  override def createAggregationBuffer(): Union = {
+    new Union(lgMaxK)
+  }
+
+  /**
+   * Update the Union instance with the HllSketch byte array obtained from the row.
+   *
+   * @param union The Union instance.
+   * @param input an input row
+   */
+  override def update(union: Union, input: InternalRow): Union = {
+    val v = child.eval(input)
+    if (v != null) {
+      child.dataType match {
+        case BinaryType =>
+          union.update(HllSketch.wrap(WritableMemory.writableWrap(v.asInstanceOf[Array[Byte]])))
+        case _ => throw new UnsupportedOperationException(
+          s"A Union instance can only be updated with a valid HllSketch byte array")
+      }
+    }
+    union
+  }
+
+  /**
+   * Merges an input Union into the union which is acting as the aggregation buffer.
+   *
+   * @param union the Union instance used to store the aggregation result.
+   * @param input an input Union instance
+   */
+  override def merge(union: Union, input: Union): Union = {
+    union.update(input.getResult)
+    union
+  }
+
+  /**
+   * Returns the estimated number of unique values derived from the merged HllSketches
+   *
+   * @param union Union instance used as an aggregation buffer
+   * @return A long value representing the number of uniques
+   */
+  override def eval(union: Union): Any = {
+    union.getResult.getEstimate.toLong
+  }

Review Comment:
   When using `.toLong` is seems that we compute the floor of the estimate. I think it is better to use `Math.round` here instead. This is also what `APPROX_COUNT_DISTINCT` does.



##########
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:
   The comment says that we serialize to an updatable byte array, but then we serialize to a compact byte array. These need to be consistent. Also I would suggest to serialize to an updatable byte array, as serde in this case should be faster.



##########
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala:
##########
@@ -1545,6 +1545,158 @@ class DataFrameAggregateSuite extends QueryTest
     )
     checkAnswer(res, Row(Array(1), Array(1)))
   }
+
+  test("SPARK-16484: hllsketch_estimate positive tests") {
+    val df1 = Seq(
+      (1, "a"), (1, "a"), (1, "a"),
+      (1, "b"),
+      (1, "c"), (1, "c"),
+      (1, "d")
+    ).toDF("id", "value")
+    df1.createOrReplaceTempView("df1")
+
+    val df2 = Seq(
+      (1, "a"),
+      (1, "c"),
+      (1, "d"), (1, "d"), (1, "d"),
+      (1, "e"), (1, "e"),
+      (1, "f")
+    ).toDF("id", "value")
+    df2.createOrReplaceTempView("df2")
+
+    // empty column test
+    val res0 = df1.withColumn("empty_col", lit(null)).groupBy("id")
+      .agg(
+        hllsketch_estimate("empty_col").as("distinct_count")
+      )
+    checkAnswer(res0, Row(1, 0))
+
+    // first test hllsketch_estimate via dataframe + sql, with and without configs
+    val res1 = df1.groupBy("id")
+      .agg(
+        count("value").as("count"),
+        hllsketch_estimate("value").as("distinct_count_1"),
+        hllsketch_estimate("value", 20, "HLL_8").as("distinct_count_2")
+      )
+    checkAnswer(res1, Row(1, 7, 4, 4))
+
+    val res2 = sql(
+      """select
+        | id,
+        | count(value) as count,
+        | hllsketch_estimate(value) as distinct_count_1,
+        | hllsketch_estimate(value, 20, 'HLL_8') as distinct_count_2
+        |from df1
+        |group by 1
+        |""".stripMargin)
+    checkAnswer(res2, Row(1, 7, 4, 4))
+
+    // next test hllsketch_binary via dataframe + sql, with and without configs
+    val df3 = df1.groupBy("id")
+      .agg(
+        count("value").as("count"),
+        hllsketch_binary("value").as("hllsketch_1"),
+        hllsketch_binary("value", 20, "HLL_8").as("hllsketch_2"),
+        hllsketch_binary("value").as("hllsketch_3")
+      )
+    df3.createOrReplaceTempView("df3")
+
+    // now test hllsketch_union_estimate via dataframe + sql, with and without configs,
+    // unioning together sketches with default, non-default and different configurations
+    val df4 = sql(
+      """select
+        | id,
+        | count(value),
+        | hllsketch_binary(value) as hllsketch_1,
+        | hllsketch_binary(value, 20, 'HLL_8') as hllsketch_2,
+        | hllsketch_binary(value, 20, 'HLL_8') as hllsketch_3
+        |from df2
+        |group by 1
+        |""".stripMargin)
+    df4.createOrReplaceTempView("df4")
+
+    val res3 = df3.union(df4).groupBy("id")
+      .agg(
+        sum("count").as("count"),
+        hllsketch_union_estimate("hllsketch_1", 20).as("distinct_count_1"),
+        hllsketch_union_estimate("hllsketch_2").as("distinct_count_2"),
+        hllsketch_union_estimate("hllsketch_3").as("distinct_count_3")
+      )
+    checkAnswer(res3, Row(1, 15, 6, 6, 6))
+
+    val res4 = sql(
+      """select
+        | id,
+        | sum(count) as count,
+        | hllsketch_union_estimate(hllsketch_1, 20) as distinct_count_1,
+        | hllsketch_union_estimate(hllsketch_2) as distinct_count_2,
+        | hllsketch_union_estimate(hllsketch_3) as distinct_count_3
+        |from (select * from df3 union all select * from df4)
+        |group by 1
+        |""".stripMargin)
+    checkAnswer(res4, Row(1, 15, 6, 6, 6))
+  }
+
+  test("SPARK-16484: hllsketch_estimate negative tests") {
+
+    val df1 = Seq(
+      (1, "a"), (1, "a"), (1, "a"),
+      (1, "b"),
+      (1, "c"), (1, "c"),
+      (1, "d")
+    ).toDF("id", "value")
+
+    // validate that the functions error out when lgConfigK < 0
+    val error0 = intercept[AnalysisException] {
+      val res = df1.groupBy("id")
+        .agg(
+          hllsketch_estimate("value", -1, "HLL_4").as("hllsketch")
+        )
+      checkAnswer(res, Nil)
+    }
+    assert(error0.toString contains "DATATYPE_MISMATCH")
+
+    val error1 = intercept[AnalysisException] {
+      val res = df1.groupBy("id")
+        .agg(
+          hllsketch_binary("value", -1, "HLL_4").as("hllsketch")
+        )
+      checkAnswer(res, Nil)
+    }
+    assert(error1.toString contains "DATATYPE_MISMATCH")
+
+    val error2 = intercept[AnalysisException] {
+      val res = df1.groupBy("id")
+        .agg(
+          hllsketch_binary("value").as("hllsketch")
+        )
+        .agg(
+          hllsketch_union_estimate("hllsketch", -1)
+        )
+      checkAnswer(res, Nil)
+    }
+    assert(error2.toString contains "DATATYPE_MISMATCH")
+
+    // validate that the functions error out with unsupported tgtHllType
+    val error3 = intercept[SparkException] {
+      val res = df1.groupBy("id")
+        .agg(
+          hllsketch_estimate("value", 12, "HLL_5").as("hllsketch")
+        )
+      checkAnswer(res, Nil)
+    }
+    assert(error3.toString contains "IllegalArgumentException")

Review Comment:
   This is also not such a nice error to have, and I believe it does not follow the structure of error messages (does not have an associated error class...). Let's discuss about this.
   
   Although this is not the right place to discuss this, I am wondering whether we really need to expose the `tgtHllType` as part of this PR. Also something to discuss.



##########
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
+  }
+
+  /** De-serializes the updateable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllSketchEstimate function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgConfigK the log-base-2 of K, where K is the number of buckets or slots for the sketch
+ * @param tgtHllType the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8)
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgConfigK, tgtHllType]) - Returns the estimated number of unique values.
+      `lgConfigK` the log-base-2 of K, where K is the number of buckets or slots for the HllSketch.
+      `tgtHllType` the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends HllSketchAggregate {
+
+  // all the constructors, case class's apply doesn't work for sql
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression, tgtHllType: Expression) = {
+    this(child, lgConfigK, tgtHllType, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int, tgtHllType: String) = {
+    this(child, Literal(lgConfigK), Literal(tgtHllType), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchEstimate =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchEstimate =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchEstimate =
+    copy(child = newFirst, lgConfigKExpression = newSecond, tgtHllTypeExpression = newThird)
+
+  // overrides for TernaryLike
+
+  override def first: Expression = child
+  override def second: Expression = lgConfigKExpression
+  override def third: Expression = tgtHllTypeExpression
+
+  // Implementations specific to HllSketchEstimate
+
+  override def prettyName: String = "hllsketch_estimate"
+
+  override def dataType: DataType = LongType
+
+  /**
+   * Returns the estimated number of unique values
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A long value representing the number of uniques
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.getEstimate.toLong
+  }
+}
+
+/**
+ * The HllSketchBinary function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the compact binary representation of the HllSketch.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgConfigK the log-base-2 of K, where K is the number of buckets or slots for the sketch
+ * @param tgtHllType the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8)
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgConfigK, tgtHllType]) - Returns the HllSketch's compact binary representation.
+      `lgConfigK` the log-base-2 of K, where K is the number of buckets or slots for the HllSketch.
+      `tgtHllType` the target type of the HllSketch to be used (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hllsketch_binary_estimate(_FUNC_(col1))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchBinary(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends HllSketchAggregate {
+
+  // all the constructors, case class's apply doesn't work for sql
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression, tgtHllType: Expression) = {
+    this(child, lgConfigK, tgtHllType, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int, tgtHllType: String) = {
+    this(child, Literal(lgConfigK), Literal(tgtHllType), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchBinary =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchBinary =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchBinary =
+    copy(child = newFirst, lgConfigKExpression = newSecond, tgtHllTypeExpression = newThird)
+
+  // overrides for TernaryLike
+
+  override def first: Expression = child
+  override def second: Expression = lgConfigKExpression
+  override def third: Expression = tgtHllTypeExpression
+
+  // Implementations specific to HllSketchBinary
+
+  override def prettyName: String = "hllsketch_binary"
+
+  override def dataType: DataType = BinaryType
+
+  /**
+   * Returns the compact byte array associated with the HllSketch
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary value which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+}
+
+/**
+ * The HllSketchUnionEstimate function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the estimated unique count from the merged HllSketches.
+ *
+ * See [[https://datasketches.apache.org/docs/HLL/HLL.html]] for more information
+ *
+ * @param child child expression against which unique counting will occur
+ * @param lgMaxK The largest maximum size for lgConfigK for the union operation.
+ */
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr[, lgMaxK]) - Returns the estimated number of unique values.
+      `lgMaxK` The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hllsketch_binary(col1))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchUnionEstimate(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // all the constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgMaxK: Expression) = {
+    this(child, lgMaxK, 0, 0)
+  }
+
+  def this(child: Expression, lgMaxK: Int) = {
+    this(child, Literal(lgMaxK), 0, 0)
+  }
+
+  // These copy constructors are repeated in every case class
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllSketchUnionEstimate = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchUnionEstimate =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllSketchUnionEstimate = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // overrides for BinaryLike
+
+  override def left: Expression = child
+  override def right: Expression = lgMaxKExpression
+
+  // Type checking
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (left.dataType, right.dataType) match {
+      case (_, NullType) =>
+        DataTypeMismatch(
+          errorSubClass = "UNEXPECTED_NULL",
+          messageParameters = Map(
+            "exprName" -> "lgMaxK"
+          )
+        )
+      case (BinaryType, IntegerType) =>
+        if (!right.foldable) {
+          DataTypeMismatch(
+            errorSubClass = "NON_FOLDABLE_INPUT",
+            messageParameters = Map(
+              "inputName" -> "lgMaxK",
+              "inputType" -> toSQLType(right.dataType),
+              "inputExpr" -> toSQLExpr(right)
+            )
+          )
+        } else if (lgMaxK <= 0L) {
+          DataTypeMismatch(
+            errorSubClass = "VALUE_OUT_OF_RANGE",
+            messageParameters = Map(
+              "exprName" -> "lgMaxK",
+              "valueRange" -> s"[0, positive]",
+              "currentValue" -> toSQLValue(lgMaxK, IntegerType)
+            )
+          )
+        } else {
+          TypeCheckSuccess
+        }
+      case _ =>
+        DataTypeMismatch(
+          errorSubClass = "HLLSKETCH_WRONG_TYPE",
+          messageParameters = Map(
+            "functionName" -> toSQLId(prettyName),
+            "expectedRight" -> toSQLType(IntegerType),
+            "actual" -> Seq(left.dataType, right.dataType)
+              .map(toSQLType).mkString(", ")
+          )
+        )
+    }
+  }
+
+  // Implementations specific to HllSketchUnionEstimate
+
+  override def prettyName: String = "hllsketch_union_estimate"
+
+  override def dataType: DataType = LongType
+
+  override def nullable: Boolean = false
+
+  /**
+   * Instantiate an Union instance using the lgMaxK param.
+   *
+   * @return an Union instance
+   */
+  override def createAggregationBuffer(): Union = {
+    new Union(lgMaxK)
+  }
+
+  /**
+   * Update the Union instance with the HllSketch byte array obtained from the row.
+   *
+   * @param union The Union instance.
+   * @param input an input row
+   */
+  override def update(union: Union, input: InternalRow): Union = {
+    val v = child.eval(input)
+    if (v != null) {
+      child.dataType match {
+        case BinaryType =>
+          union.update(HllSketch.wrap(WritableMemory.writableWrap(v.asInstanceOf[Array[Byte]])))

Review Comment:
   Why do we use `WritableMemory` here?  Wouldn't it be enough to use `Memory` (immutable memory wrapper) instead?



##########
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)

Review Comment:
   I would expect to support also floating point types and BINARY values. How come these are missing?



##########
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)
+            )
+          )

Review Comment:
   As I mention in another comment, I believe this should actually check for the correct acceptable values. If I am not mistaken the correct value range is `[4, 21]`.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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