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

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

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