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

[GitHub] [spark] RyanBerti opened a new pull request, #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

RyanBerti opened a new pull request, #40615:
URL: https://github.com/apache/spark/pull/40615

   ### What changes were proposed in this pull request?
   This PR adds a new dependency on the datasketches-java project, and provides 3 new functions which utilize Datasketches HllSketch and Union instances for unique counting and intermediate storage of HLL sketches. 
   
   
   ### Why are the changes needed?
   The existing approx_count_distinct provides unique counting functionality, but does not provide the ability to store HLL sketches for re-aggregation. After discussions with the Databricks team, we decided that integrating the Datasketches HllSketch implementation would make more sense than extending the existing approx_count_distinct functionality, given the wide use of the Datasketches library. See https://github.com/apache/spark/pull/39678 and https://github.com/RyanBerti/spark/pull/1 for more information.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, this PR introduces three new aggregate functions:
   - hllsketch_estimate
   - hllsketch_binary
   - hllsketch_union_estimate
   
   
   ### How was this patch tested?
   <WIP> I've included a simple test in the DataframeAggregateSuite; more tests to be added.
   


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


[GitHub] [spark] HyukjinKwon closed pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch
URL: https://github.com/apache/spark/pull/40615


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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1155402075


##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,120 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions

Review Comment:
   Should also add one in `pyspark.sql.connect.functions` too.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1173434755


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

Review Comment:
   Will take a look at them as soon as possible.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1516584672

   > A few tests are failing due to some connectivity issues unrelated to the changes in this PR - is there an easy way to re-run without pushing a new commit?
   
   I am not aware how to do that. @gatorsmile any ideas?


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172946445


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

Review Comment:
   I thought I had tested hll_union, but that one was missing. Will add more explicit tests for non-aggregate expressions, separate+downstream of aggregate functions.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162115988


##########
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:
   Good catch - will update.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172894027


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray

Review Comment:
   My understanding is that using the updatable representation has two benefits: 
   - Less overhead producing this representation, when compared to compact
   - The sketch can be ['wrapped' and updated directly](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/hll/BaseHllSketch.java#L245-L262), vs reconstructing the sketch on-heap 
   
   Because we're always reconstructing the sketch on-heap, we wouldn't really be benefitting from writing out the updateable representation of the sketch, so I figured writing out a smaller sketch would be preferred. I'm happy to change this to updateable if you feel strongly the output should be in this format.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1154928718


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // From here on, these are the shared default implementations for TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never return null */

Review Comment:
   Just ran a quick test, and both null and empty return 0: 
   
   `  val sketch = new HllSketch()
       assert(sketch.getEstimate == 0)
       val nullString: String = null
       sketch.update(nullString)
       assert(sketch.getEstimate == 0)`



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // From here on, these are the shared default implementations for TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never return null */

Review Comment:
   Just ran a quick test, and both null and empty return 0: 
   
   ```  val sketch = new HllSketch()
       assert(sketch.getEstimate == 0)
       val nullString: String = null
       sketch.update(nullString)
       assert(sketch.getEstimate == 0)```



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


[GitHub] [spark] HyukjinKwon commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1531272567

   Merged to master.


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181935926


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {

Review Comment:
   Migrated magic numbers + error handling function to companion object.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162115307


##########
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:
   Good catch - I initially had it returning an updateable byte array as I thought we'd be able to wrap/update that byte array after deserialization. I converted to the compact byte array as the wrap/update mechanism didn't work as expected, and assumed the smaller format would be preferred. I didn't think the overhead of serializing/deserializing one or the other would be significant. I can migrate back to writing out the updateable byte array, but heapifying the updateable byte array on deserialization. 



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1173046013


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

Review Comment:
   Updated tests to validate non-aggregate functions in both grouping and non-grouping expressions.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172908797


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        case dataType => throw new UnsupportedOperationException(
+          s"A HllSketch instance cannot be updates with a Spark ${dataType.toString} type")

Review Comment:
   I think we can keep this exception for cases where the update() is being called outside of normal Spark operations.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172907828


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)

Review Comment:
   Just found an example of using TypeCollection to define the full set of types supported by the function's first arg; will update the implementation to use that object.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172914590


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   I think I can use the [checkLgK function](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/cpc/CpcUtil.java#L34) to validate, at the point the lazy val is evaluated, and throw an IllegalArgumentException if the value is invalid.



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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1542840734

   @bersprockets good catch - I thought relying on ExpectsInputTypes would be sufficient. Looking at other example functions like ApproximatePercentile, I see that the foldable check is applied after super.checkInputDataTypes() in an overridden checkInputDataTypes() implementation. I'll follow this pattern and open up a new PR.


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162107556


##########
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:
   I think this is coming from Enum.valueOf. I could wrap the exception with a SketchesArgumentException (what gets thrown when instantiating an HllSketch with an invalid lgConfigK value)?



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


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

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1180886403


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {

Review Comment:
   no need for `else` since you throw an exception above



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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.

Review Comment:
   We could return an error in this case initially and defer the problem to the next PR if we want.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {

Review Comment:
   can you move these magic numbers to private constants in the class with comments explaining what they mean?



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1528840439

   @RyanBerti I created a PR against this PR for updating `sql-expression-schema.md` (I regenerated the golden file).
   Feel free to either merge my PR into yours, or just copy the file in my PR into your code manually.
   Just trying to help here :)
   
   My PR: https://github.com/RyanBerti/spark/pull/4


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


[GitHub] [spark] bersprockets commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "bersprockets (via GitHub)" <gi...@apache.org>.
bersprockets commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1541016431

   As a follow-up, should you add a check to ensure a foldable `lgConfigK`?:
   
   ```
   spark-sql (default)> create or replace temp view v1 as
   select * from values
   (1, 12),
   (1, 12),
   (2, 12),
   (2, 12),
   (3, 12)
   as tab(col, logk);
   Time taken: 1.665 seconds
   spark-sql (default)> select hex(hll_sketch_agg(col, logk)) from v1;
   23/05/09 16:25:25 ERROR Executor: Exception in task 4.0 in stage 0.0 (TID 4)
   java.lang.NullPointerException
   	at org.apache.spark.sql.catalyst.InternalRow$.$anonfun$getAccessor$4(InternalRow.scala:138)
   	at org.apache.spark.sql.catalyst.InternalRow$.$anonfun$getAccessor$4$adapted(InternalRow.scala:138)
   	at org.apache.spark.sql.catalyst.expressions.BoundReference.eval(BoundAttribute.scala:40)
   	at org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg.lgConfigK$lzycompute(datasketchesAggregates.scala:65)
   	at org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg.lgConfigK(datasketchesAggregates.scala:64)
   	at org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg.createAggregationBuffer(datasketchesAggregates.scala:116)
   	at org.apache.spark.sql.catalyst.expressions.aggregate.HllSketchAgg.createAggregationBuffer(datasketchesAggregates.scala:55)
   ...
   ```


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1156737021


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // 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 = {
+    // scalastyle:off caselocale
+    new HllSketch(lgConfigK, TgtHllType.valueOf(tgtHllType.toUpperCase))
+    // scalastyle:on caselocale
+  }
+
+  /**
+   * 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 = child.eval(input)
+    if (v != null) {
+      child.dataType match {
+        // Update implemented for all types supported by HllSketch
+        // Spark SQL doesn't have equivalent types for ByteBuffer or char[] so leave those out
+        case IntegerType => sketch.update(v.asInstanceOf[Int])
+        case LongType => sketch.update(v.asInstanceOf[Long])
+        case DoubleType => sketch.update(v.asInstanceOf[Double])

Review Comment:
   Ok, I've removed the DoubleType case, and added some more comments + a TODO.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172917855


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -539,6 +539,235 @@ object functions {
   def grouping_id(colName: String, colNames: String*): Column =
     grouping_id((Seq(colName) ++ colNames).map(n => Column(n)): _*)
 
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column, lgConfigK: Int, tgtHllType: String): Column =
+    Column.fn("hll_sketch_agg", e, lit(lgConfigK), lit(tgtHllType))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String, lgConfigK: Int, tgtHllType: String): Column =
+    Column.fn("hll_sketch_agg", Column(columnName), lit(lgConfigK), lit(tgtHllType))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK arg and default tgtHllType value.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column, lgConfigK: Int): Column =
+    Column.fn("hll_sketch_agg", e, lit(lgConfigK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK arg and default tgtHllType value.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String, lgConfigK: Int): Column =
+    Column.fn("hll_sketch_agg", Column(columnName), lit(lgConfigK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with default lgConfigK and tgtHllType values.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column): Column =
+    Column.fn("hll_sketch_agg", e)
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with default lgConfigK and tgtHllType values.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String): Column =
+    Column.fn("hll_sketch_agg", Column(columnName))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(e: Column, lgMaxK: Int): Column =
+    Column.fn("hll_union_agg", e, lit(lgMaxK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(columnName: String, lgMaxK: Int): Column =
+    Column.fn("hll_union_agg", Column(columnName), lit(lgMaxK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with default lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(e: Column): Column =
+    Column.fn("hll_union_agg", e)
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with default lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(columnName: String): Column =
+    Column.fn("hll_union_agg", Column(columnName))
+
+  /**
+   * Aggregate function: returns the estimated number of unique items in a group, using a
+   * Datasketches HllSketch instance configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hllsketch_estimate(columnName: String, lgConfigK: Int, tgtHllType: String): Column = {
+    Column.fn("hllsketch_estimate", Column(columnName), lit(lgConfigK), lit(tgtHllType))
+  }

Review Comment:
   I think I copy/pasted these and forgot to remove the old definitions. Removed.



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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1516899001

   @mkaravel regarding your comment about 'mixing sketches with different lgk values',[ this is the Union implementation which handles merging sketches with different configs](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/hll/Union.java#L317-L340); my assumption is that the functionality is tested and stable, but let me know if you think we should try to limit the union operation to only support sketches with the same config?


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1175541542


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray

Review Comment:
   I'm not seeing any implementation specifics [here](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/hll/HllSketch.java#L169) or within t[he default HLL type's implementation](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/hll/Hll8Array.java#L53) that differentiate whether the sketch is in compact or updateable mode. Maybe there's something in the initial wrap()? For reference, it seems like Trino [allows you to move between both representations of the sketch](https://trino.io/docs/current/language/types.html#hyperloglog), so maybe a follow-up task could add similar functionality in Spark? I'll modify to output the updateable representation for now.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162116225


##########
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:
   Another good point - will update accordingly.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162110985


##########
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:
   @dtenedor proposed we don't support floating point types in this comment above: https://github.com/apache/spark/pull/40615#discussion_r1154843262
   
   I think supporting BinaryType should be do-able, though it's underlying representation would be an Array[Byte], and during our last discussion I think the guidance was we didn't expect many use cases that would require cardinality estimation of array/map/complex types.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162128291


##########
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:
   I've removed the explicit checks in the aggregate function and instead am relying on the HllSketch/Union arg checks/exceptions.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1154926480


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // 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 = {
+    // scalastyle:off caselocale

Review Comment:
   I initially had that in-place, but removed it as I wasn't sure the enum.valueOf call would handle any non-english locales. I can update with Locale.Root though, just was unfamiliar with that implementation.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181937235


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {
+      lgConfigK
+    }
+  }
+
+  // Constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), 0, 0)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression,
+                                                 newRight: Expression): HllSketchAgg =
+    copy(child = newLeft, lgConfigKExpression = newRight)
+
+  // Overrides for TernaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgConfigKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] =
+    Seq(TypeCollection(IntegerType, LongType, StringType, BinaryType), IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  override def nullable: Boolean = false
+
+  /**
+   * Instantiate an HllSketch instance using the lgConfigK param.
+   *
+   * @return an HllSketch instance
+   */
+  override def createAggregationBuffer(): HllSketch = {
+    new HllSketch(lgConfigK)
+  }
+
+  /**
+   * 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

Review Comment:
   Updated arg references + descriptions.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1164754887


##########
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:
   Ok, I'm happy to add support for BinaryType - should be a simple update.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181005091


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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.

Review Comment:
   Actually Ryan implemented my suggestion to block merging sketches with different `lgk` by default and allow via an optional boolean parameter to merge them even if `lgk`'s are different.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1180891177


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {
+      lgConfigK
+    }
+  }
+
+  // Constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), 0, 0)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression,
+                                                 newRight: Expression): HllSketchAgg =
+    copy(child = newLeft, lgConfigKExpression = newRight)
+
+  // Overrides for TernaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgConfigKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] =
+    Seq(TypeCollection(IntegerType, LongType, StringType, BinaryType), IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  override def nullable: Boolean = false
+
+  /**
+   * Instantiate an HllSketch instance using the lgConfigK param.
+   *
+   * @return an HllSketch instance
+   */
+  override def createAggregationBuffer(): HllSketch = {
+    new HllSketch(lgConfigK)
+  }
+
+  /**
+   * 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

Review Comment:
   nit: "an" -> "An"



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {

Review Comment:
   +1



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {
+      lgConfigK
+    }
+  }
+
+  // Constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), 0, 0)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression,
+                                                 newRight: Expression): HllSketchAgg =
+    copy(child = newLeft, lgConfigKExpression = newRight)
+
+  // Overrides for TernaryLike

Review Comment:
   Seems like this is leftover. It should be "BinaryLike"



##########
sql/core/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -597,6 +597,103 @@ object functions {
     grouping_id((Seq(colName) ++ colNames).map(n => Column(n)) : _*)
   }
 
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches

Review Comment:
   I believe we are returning the updatable representation now.
   Same for the functions below.



##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,157 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions
+def hll_sketch_agg(col: "ColumnOrName", lgConfigK: Optional[int] = None) -> Column:
+    """
+    Aggregate function: returns the updatable binary representation of the Datasketches
+    HllSketch configured with lgConfigK arg.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    col : :class:`~pyspark.sql.Column` or str
+    lgConfigK : int, optional
+        The log-base-2 of K, where K is the number of buckets or slots for the HllSketch
+
+    Returns
+    -------
+    :class:`~pyspark.sql.Column`
+        The binary representation of the HllSketch.
+
+    Examples
+    --------
+    >>> df = spark.createDataFrame([1,2,2,3], "INT")
+    >>> df = df.agg(hll_sketch_estimate(hll_sketch_agg("value")).alias("distinct_cnt"))
+    >>> df.show()
+    +------------+
+    |distinct_cnt|
+    +------------+
+    |           3|
+    +------------+
+    """
+    if lgConfigK is not None:
+        return _invoke_function("hll_sketch_agg", _to_java_column(col), lgConfigK)
+    else:
+        return _invoke_function("hll_sketch_agg", _to_java_column(col))
+
+
+@try_remote_functions
+def hll_union_agg(col: "ColumnOrName", allowDifferentLgConfigK: Optional[bool] = None) -> Column:
+    """
+    Aggregate function: returns the updaable binary representation of the Datasketches

Review Comment:
   Typo: "updaable" -> "updatable"



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -539,6 +539,92 @@ object functions {
   def grouping_id(colName: String, colNames: String*): Column =
     grouping_id((Seq(colName) ++ colNames).map(n => Column(n)): _*)
 
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch

Review Comment:
   Here and below: I believe we are now returning the updatable sketch representation.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and

Review Comment:
   nit: Forgive me for making this comment (this is my math/algorithms background kicking in):
   
   "to probabilistically count": when I read this I under stand that the algorithm for computing the number of unique values is probabilistic (or randomized if you want). I think what we want to convey here is that the count is based on probabilistic analysis. How about: "to count a probabilistic approximation of the number of unique values...".



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {

Review Comment:
   +1



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181937412


##########
sql/core/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -597,6 +597,103 @@ object functions {
     grouping_id((Seq(colName) ++ colNames).map(n => Column(n)) : _*)
   }
 
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches

Review Comment:
   Good catch - updated all of the descriptions to replace 'compact' with 'updatable'.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1167155931


##########
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:
   Resolving this comment as I've added tests for all ideas listed, except for the `Merging sparse sketches with dense sketches` idea which I think we should rely on the datasketches tests to cover.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172824899


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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` (optional) The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(sketch, 12))
+      FROM (
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (1), (1), (2), (2), (3) tab(col1)
+        UNION ALL
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (4), (4), (5), (5), (6) tab(col1)
+      );
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllUnionAgg(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllUnionAgg = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllUnionAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnionAgg = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // Overrides for BinaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgMaxKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_union_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  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(Memory.wrap(v.asInstanceOf[Array[Byte]])))
+        case _ => throw new UnsupportedOperationException(
+          s"A Union instance can only be updated with a valid HllSketch byte array")

Review Comment:
   Do we need this?
   Since the aggregate expression is defined with `ExpectsInputTypes` it should never be the case that we have anything else other that a BINARY value.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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` (optional) The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(sketch, 12))
+      FROM (
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (1), (1), (2), (2), (3) tab(col1)
+        UNION ALL
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (4), (4), (5), (5), (6) tab(col1)
+      );
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllUnionAgg(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllUnionAgg = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllUnionAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnionAgg = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // Overrides for BinaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgMaxKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_union_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  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(Memory.wrap(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 an HllSketch derived from the merged HllSketches
+   *
+   * @param union Union instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(union: Union): Any = {
+    union.toCompactByteArray
+  }

Review Comment:
   Same question here: why compact representation?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -539,6 +539,235 @@ object functions {
   def grouping_id(colName: String, colNames: String*): Column =
     grouping_id((Seq(colName) ++ colNames).map(n => Column(n)): _*)
 
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column, lgConfigK: Int, tgtHllType: String): Column =
+    Column.fn("hll_sketch_agg", e, lit(lgConfigK), lit(tgtHllType))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String, lgConfigK: Int, tgtHllType: String): Column =
+    Column.fn("hll_sketch_agg", Column(columnName), lit(lgConfigK), lit(tgtHllType))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK arg and default tgtHllType value.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column, lgConfigK: Int): Column =
+    Column.fn("hll_sketch_agg", e, lit(lgConfigK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with lgConfigK arg and default tgtHllType value.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String, lgConfigK: Int): Column =
+    Column.fn("hll_sketch_agg", Column(columnName), lit(lgConfigK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with default lgConfigK and tgtHllType values.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(e: Column): Column =
+    Column.fn("hll_sketch_agg", e)
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch
+   * configured with default lgConfigK and tgtHllType values.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_sketch_agg(columnName: String): Column =
+    Column.fn("hll_sketch_agg", Column(columnName))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(e: Column, lgMaxK: Int): Column =
+    Column.fn("hll_union_agg", e, lit(lgMaxK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(columnName: String, lgMaxK: Int): Column =
+    Column.fn("hll_union_agg", Column(columnName), lit(lgMaxK))
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with default lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(e: Column): Column =
+    Column.fn("hll_union_agg", e)
+
+  /**
+   * Aggregate function: returns the compact binary representation of the Datasketches HllSketch,
+   * generated by merging previously created Datasketches HllSketch instances via a Datasketches
+   * Union instance configured with default lgMaxK arg.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hll_union_agg(columnName: String): Column =
+    Column.fn("hll_union_agg", Column(columnName))
+
+  /**
+   * Aggregate function: returns the estimated number of unique items in a group, using a
+   * Datasketches HllSketch instance configured with lgConfigK and tgtHllType args.
+   *
+   * @group agg_funcs
+   * @since 3.5.0
+   */
+  def hllsketch_estimate(columnName: String, lgConfigK: Int, tgtHllType: String): Column = {
+    Column.fn("hllsketch_estimate", Column(columnName), lit(lgConfigK), lit(tgtHllType))
+  }

Review Comment:
   Here and below (aggregate section): do we still have these functions? I believe they have changed, including their names.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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` (optional) The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(sketch, 12))
+      FROM (
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (1), (1), (2), (2), (3) tab(col1)
+        UNION ALL
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (4), (4), (5), (5), (6) tab(col1)
+      );
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllUnionAgg(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)

Review Comment:
   The union operation does not take a register size as input (like the sketch creation aggregate function). In this case the DataSketches default choice will be used (4 bits per register). I think this could be problematic in terms of user expectations. To be honest, I think that exposing the register size in the first place in the sketch creation aggregate function is too much detail, and I would always use 8 bits per register as this is the fastest choice (opting for performance vs memory consumption here).



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, LongType}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the estimated number of unique values given the binary representation
+    of a Datasketches HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hll_sketch_agg(col1)
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(child: Expression)
+  extends UnaryExpression
+    with CodegenFallback

Review Comment:
   Why no codegen? It should be very easy to implement.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        case dataType => throw new UnsupportedOperationException(
+          s"A HllSketch instance cannot be updates with a Spark ${dataType.toString} type")

Review Comment:
   Do we really need this (see my other comment). Shouldn't we just list the allowed input data types and stop there?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)

Review Comment:
   The aggregate expression is defined with `ExpectsInputTypes`. Why do we add `AnyDataType` here and then check the type, instead of explicitly defining the allowed data types here?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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.

Review Comment:
   I am a bit confused here: let's say that we have a column of sketches with lgk values from 9 to 12 and you pass a value that is greater than 9 (say 11). Then the resulting sketch will have a lgk 9. In my opinion hiding this under the hood can be problematic. In the least the expression description should explain what happens. Ideally, we should not merge sketches with different lgk values by default (default call to the function) and we should have a separate API that allows to force downsampling and loss of sketch precision.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, LongType}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the estimated number of unique values given the binary representation
+    of a Datasketches HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hll_sketch_agg(col1)
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(child: Expression)
+  extends UnaryExpression
+    with CodegenFallback
+    with ExpectsInputTypes
+    with NullIntolerant {
+
+  override protected def withNewChildInternal(newChild: Expression): HllSketchEstimate =
+    copy(child = newChild)
+
+  override def prettyName: String = "hll_sketch_estimate"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
+
+  override def dataType: DataType = LongType
+
+  override def nullSafeEval(input: Any): Any = {
+    val buffer = input.asInstanceOf[Array[Byte]]
+    Math.round(HllSketch.heapify(Memory.wrap(buffer)).getEstimate)
+  }
+}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(left, right) - Merges two binary representations of Datasketches HllSketch objects,
+    using a Datasketches Union object configured with an lgMaxK equal to the min of the
+    HllSketch object's lgConfigK values """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(hll_sketch_agg(col1), hll_sketch_agg(col1)))
+      FROM VALUES (1, 4), (1, 4), (2, 5), (2, 5), (3, 6) tab(col1, col2);
+       6
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllUnion(left: Expression, right: Expression)
+  extends BinaryExpression
+    with CodegenFallback
+    with ExpectsInputTypes
+    with NullIntolerant {
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnion = copy(left = newLeft, right = newRight)
+
+  override def prettyName: String = "hll_union"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, BinaryType)
+
+  override def dataType: DataType = BinaryType
+
+  override def nullSafeEval(value1: Any, value2: Any): Any = {
+    val sketch1 = HllSketch.heapify(Memory.wrap(value1.asInstanceOf[Array[Byte]]))
+    val sketch2 = HllSketch.heapify(Memory.wrap(value2.asInstanceOf[Array[Byte]]))
+    val union = new Union(Math.min(sketch1.getLgConfigK, sketch2.getLgConfigK))
+    union.update(sketch1)
+    union.update(sketch2)
+    union.getResult.toUpdatableByteArray

Review Comment:
   Here the merge operation is returned as an updatable sketch. The aggregate version returns a compact sketch.
   I think they should behave the same in terms of the kind of representation returned, and as I point out in my comment in aggregate expressions, I think it is best to use the updatable representation.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, LongType}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the estimated number of unique values given the binary representation
+    of a Datasketches HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hll_sketch_agg(col1)
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(child: Expression)
+  extends UnaryExpression
+    with CodegenFallback
+    with ExpectsInputTypes
+    with NullIntolerant {
+
+  override protected def withNewChildInternal(newChild: Expression): HllSketchEstimate =
+    copy(child = newChild)
+
+  override def prettyName: String = "hll_sketch_estimate"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
+
+  override def dataType: DataType = LongType
+
+  override def nullSafeEval(input: Any): Any = {
+    val buffer = input.asInstanceOf[Array[Byte]]
+    Math.round(HllSketch.heapify(Memory.wrap(buffer)).getEstimate)
+  }
+}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(left, right) - Merges two binary representations of Datasketches HllSketch objects,
+    using a Datasketches Union object configured with an lgMaxK equal to the min of the
+    HllSketch object's lgConfigK values """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(hll_sketch_agg(col1), hll_sketch_agg(col1)))
+      FROM VALUES (1, 4), (1, 4), (2, 5), (2, 5), (3, 6) tab(col1, col2);
+       6
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllUnion(left: Expression, right: Expression)
+  extends BinaryExpression
+    with CodegenFallback
+    with ExpectsInputTypes
+    with NullIntolerant {
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnion = copy(left = newLeft, right = newRight)
+
+  override def prettyName: String = "hll_union"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, BinaryType)
+
+  override def dataType: DataType = BinaryType
+
+  override def nullSafeEval(value1: Any, value2: Any): Any = {
+    val sketch1 = HllSketch.heapify(Memory.wrap(value1.asInstanceOf[Array[Byte]]))
+    val sketch2 = HllSketch.heapify(Memory.wrap(value2.asInstanceOf[Array[Byte]]))
+    val union = new Union(Math.min(sketch1.getLgConfigK, sketch2.getLgConfigK))

Review Comment:
   This points back to the same issue I pointed out for the aggregate functions.
   We downsample one of the sketches under the hood without the user knowing that. I think we should only allow merging when we have sketches with the same lgk value (if it is really important have a another API/overload for merging sketches with different lgk values).



##########
sql/core/src/test/resources/sql-functions/sql-expression-schema.md:
##########
@@ -422,4 +422,4 @@
 | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()') | struct<xpath(<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>, a/b/text()):array<string>> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_long(<a><b>1</b><b>2</b></a>, sum(a/b)):bigint> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_short(<a><b>1</b><b>2</b></a>, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |

Review Comment:
   How come the two new scalar functions do not appear here?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -507,6 +507,9 @@ object FunctionRegistry {
     expression[RegrSlope]("regr_slope"),
     expression[RegrIntercept]("regr_intercept"),
     expression[Mode]("mode"),
+    expression[HllSketchEstimate]("hllsketch_estimate"),

Review Comment:
   Why do we have this?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   I know that in the previous review round you mentioned that you just want to propagate the errors from the DataSketches library and do not "polish" them. However, I believe the norm now is that we return nice error messages that are actionable and within the new error framework.
   In that respect I think we should actually check the lgk input values for correctness and throw a proper error. Same for the register size (although I think it should go away from the API wherever it appears).



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datasketchesExpressions.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, LongType}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the estimated number of unique values given the binary representation
+    of a Datasketches HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(hll_sketch_agg(col1)
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllSketchEstimate(child: Expression)
+  extends UnaryExpression
+    with CodegenFallback
+    with ExpectsInputTypes
+    with NullIntolerant {
+
+  override protected def withNewChildInternal(newChild: Expression): HllSketchEstimate =
+    copy(child = newChild)
+
+  override def prettyName: String = "hll_sketch_estimate"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
+
+  override def dataType: DataType = LongType
+
+  override def nullSafeEval(input: Any): Any = {
+    val buffer = input.asInstanceOf[Array[Byte]]
+    Math.round(HllSketch.heapify(Memory.wrap(buffer)).getEstimate)
+  }
+}
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(left, right) - Merges two binary representations of Datasketches HllSketch objects,
+    using a Datasketches Union object configured with an lgMaxK equal to the min of the
+    HllSketch object's lgConfigK values """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(hll_sketch_agg(col1), hll_sketch_agg(col1)))
+      FROM VALUES (1, 4), (1, 4), (2, 5), (2, 5), (3, 6) tab(col1, col2);
+       6
+  """,
+  group = "misc_funcs",
+  since = "3.5.0")
+case class HllUnion(left: Expression, right: Expression)
+  extends BinaryExpression
+    with CodegenFallback

Review Comment:
   Same here.



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

Review Comment:
   I believe you are not done with the tests yet, but please let me make an observation: all positive queries below involve a grouping aggregation. I believe we should also have tests with non-grouping aggregations.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray

Review Comment:
   What is the logic behind the choice to return a compact representation?
   Why not stick with the updatable one?



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172909432


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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` (optional) The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(sketch, 12))
+      FROM (
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (1), (1), (2), (2), (3) tab(col1)
+        UNION ALL
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (4), (4), (5), (5), (6) tab(col1)
+      );
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllUnionAgg(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllUnionAgg = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllUnionAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnionAgg = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // Overrides for BinaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgMaxKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_union_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  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(Memory.wrap(v.asInstanceOf[Array[Byte]])))
+        case _ => throw new UnsupportedOperationException(
+          s"A Union instance can only be updated with a valid HllSketch byte array")

Review Comment:
   I think we can keep this exception for cases where the update() is being called outside of normal Spark operations.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1173440344


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray

Review Comment:
   If my understanding of how the DataSketches library works is correct, reconstructing a sketch from an updatable representation is faster compared to reconstructing it from the contact representation. This is my motivation here.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181007295


##########
sql/core/src/test/resources/sql-functions/sql-expression-schema.md:
##########
@@ -422,4 +422,4 @@
 | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()') | struct<xpath(<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>, a/b/text()):array<string>> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_long(<a><b>1</b><b>2</b></a>, sum(a/b)):bigint> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_short(<a><b>1</b><b>2</b></a>, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |

Review Comment:
   Could you give it another try?
   You have updated the examples, and my hope is that it will work now.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181935592


##########
sql/core/src/test/resources/sql-functions/sql-expression-schema.md:
##########
@@ -422,4 +422,4 @@
 | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()') | struct<xpath(<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>, a/b/text()):array<string>> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_long(<a><b>1</b><b>2</b></a>, sum(a/b)):bigint> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_short(<a><b>1</b><b>2</b></a>, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |

Review Comment:
   Thank you for the help with this one!



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1502323531

   Hi @mkaravel thank you for the review! I'll respond to your comments in-line.
   
   >more in favor of an aggregate function that merges sketches and returns the merged sketch


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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1154928718


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // From here on, these are the shared default implementations for TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never return null */

Review Comment:
   Just ran a quick test, and both null and empty return 0: 
   
   ```  
       val sketch = new HllSketch()
       assert(sketch.getEstimate == 0)
       val nullString: String = null
       sketch.update(nullString)
       assert(sketch.getEstimate == 0)
   ```



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1156738385


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.BoundReference
+import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {
+
+  def simulateUpdateMerge[T](dataType: DataType, input: Seq[Any], numSketches: Integer = 5):
+    (Long, NumericRange[Long]) = {
+
+    // create a map of agg function instances
+    val aggFunctionMap = Seq.tabulate(numSketches)(index => {
+      val sketch = HllSketchEstimate(BoundReference(0, dataType, nullable = true))
+      index -> (sketch, sketch.createAggregationBuffer())
+    }).toMap
+
+    // randomly update agg function instances
+    input.map(value => {
+      val (aggFunction, aggBuffer) = aggFunctionMap(Random.nextInt(numSketches))
+      aggFunction.update(aggBuffer, InternalRow(value))
+    })
+
+    def serializeDeserialize(tuple: (HllSketchEstimate, HllSketch)):
+      (HllSketchEstimate, HllSketch) = {
+      val (agg, buf) = tuple
+      val serialized = agg.serialize(buf)
+      (agg, agg.deserialize(serialized))
+    }
+
+    // simulate serialization -> deserialization -> merge
+    val mapValues = aggFunctionMap.values
+    val (mergedAgg, mergedBuf) = mapValues.tail.foldLeft(mapValues.head)((prev, cur) => {
+      val (prevAgg, prevBuf) = serializeDeserialize(prev)
+      val (_, curBuf) = serializeDeserialize(cur)
+
+      (prevAgg, prevAgg.merge(prevBuf, curBuf))
+    })
+
+    (mergedAgg.eval(mergedBuf).asInstanceOf[Long],
+    mergedBuf.getLowerBound(3).toLong to mergedBuf.getUpperBound(3).toLong)

Review Comment:
   I'm allowing tests to pass if the estimate is correct to within 3 std deviations. This mimics the HyperLogLogPlusPlus test harness, though I wish there was a better mechanism to ensure error is less than what's expected based on the HllSketch lgConfigK value. I've read through https://datasketches.apache.org/docs/HLL/HLL.html a few times and it's not clear to me how to derive expected error from lgConfigK. I'll continue to look around in the datasketched library.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1500518140

   @dtenedor FYI, I updated the tests and am just missing one for empty input table, and one for merging sparse/dense sketches. Once I get the build to be green, I'm going to remove the WIP tag from the PR and send an e-mail back on that initial spark-dev thread (or maybe start a new thread) letting everyone know that the implementation is open for review. I think renaming functions is still do-able at this point, so let me know if you'd like to setup another sync to discuss updated function names?


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


[GitHub] [spark] gatorsmile commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "gatorsmile (via GitHub)" <gi...@apache.org>.
gatorsmile commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1516614822

   Are you able to see the button "Re-run all jobs" ?
   <img width="1206" alt="image" src="https://user-images.githubusercontent.com/11567269/233427975-c8ecf0cd-d1c2-43cf-b367-1183c8911d44.png">
   


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172916234


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -507,6 +507,9 @@ object FunctionRegistry {
     expression[RegrSlope]("regr_slope"),
     expression[RegrIntercept]("regr_intercept"),
     expression[Mode]("mode"),
+    expression[HllSketchEstimate]("hllsketch_estimate"),

Review Comment:
   Good catch - I thought I had removed all of the old references. Removed.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1167156475


##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,120 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions

Review Comment:
   Resolving this as I found that there's tests ensuring the connect functions match the sql functions.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1154928718


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // From here on, these are the shared default implementations for TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never return null */

Review Comment:
   Just ran a quick test, and both null and empty return 0: 



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1156581221


##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,120 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions

Review Comment:
   Ok - thanks for calling that out, will do!



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1173416750


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   I think it adds significant cognitive overhead and does not offer much (in my opinion): this is about the internal representation of the sketch and not does not affect precision. The eventual goal when computing sketches is computing NDV estimates. How I get to these estimates matters less I believe, as long as I maintain the precision that I want.
   
   Also if we are to add it to the `hll_sketch_agg` why not also add it to `hll_union_agg` and `hll_union` for symmetry? And this is where I feel the entire API becomes too complicated (see also the discussion in another comment about mixing `lgk` values).
   
   In general my approach on sketches as an imaginary user is the following:
   
   Context:
   * _I have data that I want to compute the NDVs for. Exact NDV computation is costly and not necessary for my application, so I am okay if I have a way to compute approximate NDVs if they are faster._
   * _This is not an one-off operation for me: I want to be able to somehow update my approximate NDVs as more data gets it. I also want to combine NDVs from different sources, or across sources and group them together according to my application needs._
   
   _I bump onto the Spark documentation and see that Spark offers this amazing tool called HLL sketches. What is really a sketch? It's a binary value that internally encodes the number of NDVs in my data with some precision. I also see that Spark allows me to combine two or more sketches into one, and that from a sketch I can get an NDV estimate by calling a function. Can I control the precision? Yes, but for more precision I need more space, that's the trade-off I have to live with. Do I need to know the internal encoding to use this in my application? Not really. As long as Spark gives me a good API to manage my sketches (which it does: I can create sketches, I can merge them, I can get the NDV estimate any time I want) I do not really care how the sketch is encoded. I do not want it to be slow for sure, because I may need to do this for massive amounts of data._
   
   The only users that would want to know the internal encoding and the number of bits per register are those that might want to balance between the size of the sketch and it's runtime efficiency. We are talking about really fine tuning here, where one tries to balance between space requirements for the sketches in terms of storage, precision, runtime performance, and stress on the system while executing queries. I am not expecting the power users that I am describing here to be many. And if they exist, we can always add overloads that would allow the manipulation of the register size.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1164399489


##########
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:
   Regarding BINARY values: I think there is a small confusion here: in Spark we have STRING, BINARY, ARRAY, STRUCT, and MAP types. STRING and BINARY are basically arrays of characters/bytes and they are variable size data types. They are not nested, and they are essentially the same (STRINGs have additional semantics as to what should be stored, like UTF8 characters and more). ARRAY, STRUCT, and MAP are nested data types that depend on other data types (the elements of the these nested structures). We cannot really fully define them without knowing the nested types. ARRAY is variable size, STRUCT can be either fixed size or variable size (depending on the type of elements in the struct) and MAP is also variable size.
   
   All these from the point of view of types. Then we get to the representation of those types. STRING is mapped to the `UTF8String` class as the underlying "physical" type, and BINARY is mapped to an array of bytes (what you refer to as `Array[Byte]`. BINARY is not treated as an ARRAY in the Spark type system, but rather as a single non-nested type whose "physical" representation is a Scala/Java array of bytes. So in that respect it is not any different from STRING really.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1162104493


##########
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:
   I'm leaning towards removing this value check, and instead relying on the exception thrown at the time of HllSketch initialization. This would make it so we're not replicating HllSketch checks, and only validating that the datatypes being provided to the function are expected. 



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172944802


##########
sql/core/src/test/resources/sql-functions/sql-expression-schema.md:
##########
@@ -422,4 +422,4 @@
 | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()') | struct<xpath(<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>, a/b/text()):array<string>> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_long(<a><b>1</b><b>2</b></a>, sum(a/b)):bigint> |
 | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | struct<xpath_short(<a><b>1</b><b>2</b></a>, sum(a/b)):smallint> |
-| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |

Review Comment:
   [I've re-run the command described here](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala#L30-L41) a few times and it doesn't seem to update this file to include the new functions.. not sure why.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172909704


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray
+  }
+
+  /** Convert the underlying HllSketch into an updatable byte array  */
+  override def serialize(sketch: HllSketch): Array[Byte] = {
+    sketch.toUpdatableByteArray
+  }
+
+  /** De-serializes the updatable byte array into a HllSketch instance */
+  override def deserialize(buffer: Array[Byte]): HllSketch = {
+    HllSketch.heapify(buffer)
+  }
+}
+
+/**
+ * The HllUnionAgg function ingests and merges Datasketches HllSketch
+ * instances previously produced by the HllSketchBinary function, and
+ * outputs the merged HllSketch.
+ *
+ * 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` (optional) The largest maximum size for lgConfigK for the union operation.""",
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(sketch, 12))
+      FROM (
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (1), (1), (2), (2), (3) tab(col1)
+        UNION ALL
+        SELECT hll_sketch_agg(col1) as sketch FROMVALUES (4), (4), (5), (5), (6) tab(col1)
+      );
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllUnionAgg(
+    child: Expression,
+    lgMaxKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[Union] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Union config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgMaxK: Int = lgMaxKExpression.eval().asInstanceOf[Int]
+
+  // 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int):
+  HllUnionAgg = copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllUnionAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression):
+  HllUnionAgg = copy(child = newLeft, lgMaxKExpression = newRight)
+
+  // Overrides for BinaryLike
+
+  override def left: Expression = child
+
+  override def right: Expression = lgMaxKExpression
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_union_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, IntegerType)
+
+  override def dataType: DataType = BinaryType
+
+  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(Memory.wrap(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 an HllSketch derived from the merged HllSketches
+   *
+   * @param union Union instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(union: Union): Any = {
+    union.toCompactByteArray
+  }

Review Comment:
   See my comment above, happy to change if you feel strongly.



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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1530822680

   @mkaravel @dtenedor Finally got all the tests passing, thanks for all your help! Think I covered all of the most recent review comments, let me know if you need anything else from me for the merge?


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181936673


##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,157 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions
+def hll_sketch_agg(col: "ColumnOrName", lgConfigK: Optional[int] = None) -> Column:
+    """
+    Aggregate function: returns the updatable binary representation of the Datasketches
+    HllSketch configured with lgConfigK arg.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    col : :class:`~pyspark.sql.Column` or str
+    lgConfigK : int, optional
+        The log-base-2 of K, where K is the number of buckets or slots for the HllSketch
+
+    Returns
+    -------
+    :class:`~pyspark.sql.Column`
+        The binary representation of the HllSketch.
+
+    Examples
+    --------
+    >>> df = spark.createDataFrame([1,2,2,3], "INT")
+    >>> df = df.agg(hll_sketch_estimate(hll_sketch_agg("value")).alias("distinct_cnt"))
+    >>> df.show()
+    +------------+
+    |distinct_cnt|
+    +------------+
+    |           3|
+    +------------+
+    """
+    if lgConfigK is not None:
+        return _invoke_function("hll_sketch_agg", _to_java_column(col), lgConfigK)
+    else:
+        return _invoke_function("hll_sketch_agg", _to_java_column(col))
+
+
+@try_remote_functions
+def hll_union_agg(col: "ColumnOrName", allowDifferentLgConfigK: Optional[bool] = None) -> Column:
+    """
+    Aggregate function: returns the updaable binary representation of the Datasketches

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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1515134305

   A few tests are failing due to some connectivity issues unrelated to the changes in this PR - is there an easy way to re-run without pushing a new commit?


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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1173440344


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, TgtHllType, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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` (optional) the log-base-2 of K, with K = the number of buckets for the HllSketch.
+      `tgtHllType` (optional) the target type of the HllSketch (HLL_4, HLL_6, HLL_8). """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col1, 12, 'HLL_4'))
+      FROM VALUES (1), (1), (2), (2), (3) tab(col1);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    tgtHllTypeExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with TernaryLike[Expression] with ExpectsInputTypes {
+
+  // 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 = try {
+    TgtHllType.valueOf(third.eval().asInstanceOf[UTF8String].toString.toUpperCase(Locale.ROOT))
+  } catch {
+    case _: IllegalArgumentException =>
+      throw new SketchesArgumentException("Invalid tgtHllType value")
+  }
+
+  // Constructors
+
+  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.toString), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), Literal(HllSketch.DEFAULT_HLL_TYPE.toString), 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)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newFirst: Expression,
+                                              newSecond: Expression,
+                                              newThird: Expression): HllSketchAgg =
+    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
+
+  // Overrides for TypedImperativeAggregate
+
+  override def prettyName: String = "hll_sketch_agg"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, StringType)
+
+  override def dataType: DataType = BinaryType
+
+  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 BinaryType => sketch.update(v.asInstanceOf[Array[Byte]])
+        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)
+  }
+
+  /**
+   * Returns an HllSketch derived from the input column or expression
+   *
+   * @param sketch HllSketch instance used as an aggregation buffer
+   * @return A binary sketch which can be evaluated or merged
+   */
+  override def eval(sketch: HllSketch): Any = {
+    sketch.toCompactByteArray

Review Comment:
   If my understanding of how the DataSketches library works is correct, reconstructing a sketch from an updatable representation is faster/simpler compared to reconstructing it from the contact representation. This is my motivation here.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1175544253


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   Ok, I'll go ahead and remove tgtHllType as a parameter. I was of the mindset that the Spark API should match the Datasketches HllSketch API, but we can slim down the params supported by Spark for ease of use.



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1528267013

   @RyanBerti nit in the PR description:
   > Yes, this PR introduces two new aggregate functions, and two new non-aggregate functions:
   
   Maybe use "scalar" instead of "non-aggregate".


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181937012


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {
+      lgConfigK
+    }
+  }
+
+  // Constructors
+
+  def this(child: Expression) = {
+    this(child, Literal(HllSketch.DEFAULT_LG_K), 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Expression) = {
+    this(child, lgConfigK, 0, 0)
+  }
+
+  def this(child: Expression, lgConfigK: Int) = {
+    this(child, Literal(lgConfigK), 0, 0)
+  }
+
+  // Copy constructors required by ImperativeAggregate
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): HllSketchAgg =
+    copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): HllSketchAgg =
+    copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  override protected def withNewChildrenInternal(newLeft: Expression,
+                                                 newRight: Expression): HllSketchAgg =
+    copy(child = newLeft, lgConfigKExpression = newRight)
+
+  // Overrides for TernaryLike

Review Comment:
   Removed in favor of just referencing right/left directly.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181936057


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and
+ * outputs the 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
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr, lgConfigK) - Returns the HllSketch's updateable binary representation.
+      `lgConfigK` (optional) the log-base-2 of K, with K is the number of buckets or
+      slots for the HllSketch. """,
+  examples = """
+    Examples:
+      > SELECT hll_sketch_estimate(_FUNC_(col, 12)) FROM VALUES (1), (1), (2), (2), (3) tab(col);
+       3
+  """,
+  group = "agg_funcs",
+  since = "3.5.0")
+// scalastyle:on line.size.limit
+case class HllSketchAgg(
+    child: Expression,
+    lgConfigKExpression: Expression,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0)
+  extends TypedImperativeAggregate[HllSketch] with BinaryLike[Expression] with ExpectsInputTypes {
+
+  // Hllsketch config - mark as lazy so that they're not evaluated during tree transformation.
+
+  lazy val lgConfigK: Int = {
+    val lgConfigK = lgConfigKExpression.eval().asInstanceOf[Int]
+    // can't use HllUtil.checkLgK so replicate the check
+    if (lgConfigK < 4 || lgConfigK > 21) {
+      throw new SketchesArgumentException("Invalid lgConfigK value")
+    } else {

Review Comment:
   Removed in updated companion object implementation.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1156583726


##########
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:
   (Brainstorming some more tests, just making some notes here)
   - Merging sparse sketches with dense sketches
   - Writing / merging a few different combinations of sketch type / lgConfigK



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1156736796


##########
python/pyspark/sql/functions.py:
##########
@@ -10113,6 +10113,120 @@ def unwrap_udt(col: "ColumnOrName") -> Column:
     return _invoke_function("unwrap_udt", _to_java_column(col))
 
 
+@try_remote_functions

Review Comment:
   @HyukjinKwon I've added the functions as requested - could you take a quick look and make sure they look good to you?



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


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

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1154843262


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // 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 = {
+    // scalastyle:off caselocale
+    new HllSketch(lgConfigK, TgtHllType.valueOf(tgtHllType.toUpperCase))
+    // scalastyle:on caselocale
+  }
+
+  /**
+   * 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 = child.eval(input)
+    if (v != null) {
+      child.dataType match {
+        // Update implemented for all types supported by HllSketch
+        // Spark SQL doesn't have equivalent types for ByteBuffer or char[] so leave those out
+        case IntegerType => sketch.update(v.asInstanceOf[Int])
+        case LongType => sketch.update(v.asInstanceOf[Long])
+        case DoubleType => sketch.update(v.asInstanceOf[Double])

Review Comment:
   I don't think it is a good idea to support IEEE floating-point input types for these functions. There is imprecision when comparing them to each other which can lead to instability in the results. By the same token, performing a GROUP BY operation by floating-point values is not a good idea. For reference, see these function definitions which do not include floating-point input types [1].
   
   On the other hand, I don't see any reason why we can't support DecimalType, Datetime or Interval types (we can leave a TODO comment to support those in another PR if we want).
   
   [1] https://github.com/google/zetasql/blob/master/docs/hll_functions.md



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // 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 = {
+    // scalastyle:off caselocale

Review Comment:
   should we just do `toUpperCase(Locale.Root)` instead of disabling this style check?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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 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.expressions.{Expression, ExpressionDescription}
+import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.types.{BinaryType, DataType, DoubleType, IntegerType, LongType, 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 UnaryLike[Expression] {
+
+  // These are used as params when instantiating the HllSketch object
+  // and are set by the case classes' args that extend this trait
+
+  def lgConfigK: Int
+  def tgtHllType: String
+
+  // From here on, these are the shared default implementations for TypedImperativeAggregate
+
+  /** Aggregate functions which utilize HllSketch instances should never return null */

Review Comment:
   what if the input is null or empty? Is they result simply zero?



##########
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:
   we are going to need more test cases :) some ideas:
   
   * empty input table
   * use SQL to query the aggregate functions
   * explicit regular integers vs. long integers in the input table
   * minimum and maximum supported values for each data type



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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
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: 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


[GitHub] [spark] dtenedor commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1531802864

   @RyanBerti this is super awesome. Just wanted to thank you again for working on this. I'm sure lots of users will appreciate it.


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1181936821


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/datasketchesAggregates.scala:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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 org.apache.datasketches.SketchesArgumentException
+import org.apache.datasketches.hll.{HllSketch, Union}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal}
+import org.apache.spark.sql.catalyst.trees.BinaryLike
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, BooleanType, DataType, IntegerType, LongType, StringType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+/**
+ * The HllSketchAgg function utilizes a Datasketches HllSketch instance to
+ * probabilistically count the number of unique values in a given column, and

Review Comment:
   Updated comment! Thanks for helping me clarify.



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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1531857727

   @dtenedor thanks for all the help, excited to be able to utilize sketches natively in Spark!


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172929823


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   Looks like HllUtil isn't available, so I've replicated the check.



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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1523694528

   FYI it looks like [another similar implementation of a Datasketches/Spark integration already exists in its own repo ](https://github.com/Gelerion/spark-sketches/tree/spark-3.0)- I've invited the owner of that implementation to provide feedback on this PR.


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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1523999640

   @mkaravel I've updated the implementation based on your review comments. We're now returning the updatable binary representation, no longer support the tgtHllType parameter, and defer initialization of the Union instance until we've ingested the first HllSketch such that we can throw an exception when union/sketch lgConfigKs don't match. Let me know when you've had chance to re-review?


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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1164392758


##########
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 am okay with leaving out floating point values for now. I personally do not see a problem supporting them, but I do not have a strong opinion.



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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1509053727

   Hello @mkaravel ! 
   
   I've updated the PR to provide the following functions:
   
   Aggregate functions:
   - hll_sketch_agg(IntegerType|LongType|StringType|BinaryType) -> BinaryType
   - hll_union_agg(BinaryType) -> BinaryType
   
   Scalar functions
   - hll_sketch_estimate(BinaryType) -> LongType
   - hll_union(BinaryType, BinaryType) -> BinaryType
   
   Naming wise, I felt it was valuable to keep the function names aligned with the Datasketches objects they utilize, and be explicit about the operation being applied. Hopefully these function names are a good middle ground for us? I'll continue working on getting all the tests to pass, and then open the PR up for wide review.
   
   


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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1505627742

   > Hi @mkaravel thank you for the review! I'll respond to your comments in-line.
   > 
   > > more in favor of an aggregate function that merges sketches and returns the merged sketch
   > 
   > I'm not opposed to building out an agg function that merges sketches without estimating the cardinality; I think this would be beneficial for multi-stage re-aggregations. I think this topic (unfortunately) begs the question of function naming, and here's the resultant naming scheme I'd propose:
   > 
   > Aggregate functions:
   > 
   > * HllSketch (IntegerType|LongType|StringType|...) -> BinaryType
   > * HllUnion (BinaryType) -> BinaryType
   > 
   > Normal functions
   > 
   > * HllSketchEstimate (BinaryType) -> LongType
   > 
   > I think this API is simple and the lack of an aggregate function that returns the estimated cardinality is fine. What do you think?
   
   Semantically I like this API. There is one more scalar expression that I would add, it's purpose being to merge two sketches. Let me give some more details below.
   
   In terms of naming I am more in favor of dropping the "Sketch" part from the names and using an underscore to separate the "Hll" part from the rest. More specifically this is what I would choose (this is a personal opinion, and I am not reflecting opinions of anybody else other than myself):
   
   Aggregate functions:
   
   * hll_collect (IntegerType|LongType|StringType|BinaryType|...) -> BinaryType
   * {hll_merge_agg, hll_union_agg}(BinaryType) -> BinaryType
   
   Scalar functions:
   * {hll_merge, hll_union}(BinaryType, BinaryType) -> BinaryType
   * hll_estimate(BinaryType) -> LongType
   
   Above I propose hll_merge* or hll_union* for the functions that perform the union (or merge operation) on sketches. I have a slight preference for the "merge" versions.


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


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

Posted by "mkaravel (via GitHub)" <gi...@apache.org>.
mkaravel commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1517360699

   > @mkaravel regarding your comment about 'mixing sketches with different lgk values',[ this is the Union implementation which handles merging sketches with different configs](https://github.com/apache/datasketches-java/blob/master/src/main/java/org/apache/datasketches/hll/Union.java#L317-L340); my assumption is that the functionality is tested and stable, but let me know if you think we should try to limit the union operation to only support sketches with the same config?
   
   I am not questioning the correctness of the DataSketches implementation.
   
   My concern is accidental mistakes that can happen if the user does not pay attention to the `lgk` values of the input sketches. I would argue that merging two sketches with different `lgk` values is "advanced" usage of sketches, and the user should be aware that mixing such sketches comes with caveats (loosing precision with respect to the sketch with higher `lgk`). The current API hides this complexity and caveats.
   
   Let's consider another alternative (I want your opinion on this): Let's say we have two overloads (we can extend this to the aggregate version)
   * hll_merge(sketch1: BinaryType, sketch2: BinaryType)
   * hll_merge(sketch1: BinaryType, sketch2: BinaryType, allowDifferentLgKs: BooleanType)
   
   The first errors out if the `lgk` values are different.
   The second errors out if the `lgk` values are different and `allowDifferentLgKs` is `false`. However, if `allowDifferentLgk` is `true` then the second overload behaves as your current implementation.
   
   Clearly, I am talking about adding a third boolean argument, with the default value being `false`. With these two overloads, if the user tries to merge two sketches with different precision, the query will fail. If they really need to merge them, they have the opportunity to do that by means of  setting the third argument to `true`, and it will not happen accidentally without them noticing it and being proactive about the loss of precision. A nice error message for the "first overload" and good documentation will make it very clear what is going on.


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


[GitHub] [spark] RyanBerti commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1520502797

   >about adding a third boolean argument, with the default value being false


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


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

Posted by "RyanBerti (via GitHub)" <gi...@apache.org>.
RyanBerti commented on code in PR #40615:
URL: https://github.com/apache/spark/pull/40615#discussion_r1172949405


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DatasketchesHllSketchSuite.scala:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 scala.collection.immutable.NumericRange
+import scala.util.Random
+
+import org.apache.datasketches.hll.HllSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, HllSketchEstimate}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, LongType, StringType}
+import org.apache.spark.unsafe.types.UTF8String
+
+
+class DatasketchesHllSketchSuite extends SparkFunSuite {

Review Comment:
   > Same for the register size (although I think it should go away from the API wherever it appears).
   
   Looks like [the Druid implementation supports specifying tgtHllType](https://druid.apache.org/docs/latest/development/extensions-core/datasketches-hll.html), though [the original Hive UDFs don't](https://datasketches.apache.org/docs/HLL/HllHiveUDFs.html). Can you clarify why you don't think register size should be configurable in this implementation? 



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


[GitHub] [spark] Gelerion commented on pull request #40615: [SPARK-16484][SQL] Add support for Datasketches HllSketch

Posted by "Gelerion (via GitHub)" <gi...@apache.org>.
Gelerion commented on PR #40615:
URL: https://github.com/apache/spark/pull/40615#issuecomment-1528770943

   While browsing the dev-mailing list, I came across this PR and I am excited to see that data-sketches will be built-in in Spark. Interestingly, a year ago, I created a similar [project](https://github.com/Gelerion/spark-sketches) that adds sketch support for Spark versions 2.4 to 3.4. Even the implementation path is very similar :)


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