This is an automated email from the ASF dual-hosted git repository.

jmalkin pushed a commit to branch kll_functions
in repository https://gitbox.apache.org/repos/asf/datasketches-spark.git


The following commit(s) were added to refs/heads/kll_functions by this push:
     new 7739fa0  Add more functions, still need to add tests
7739fa0 is described below

commit 7739fa04415ee7de3a869d35564c33645f0fbf44
Author: Jon <[email protected]>
AuthorDate: Thu Mar 6 00:32:24 2025 -0800

    Add more functions, still need to add tests
---
 .../sql/datasketches/kll/KllFunctionRegistry.scala | 14 +++-
 .../kll/expressions/KllDoublesSketchGetK.scala     | 82 ++++++++++++++++++++++
 .../KllDoublesSketchGetNumRetained.scala           | 82 ++++++++++++++++++++++
 .../expressions/KllDoublesSketchGetPmfCdf.scala    |  1 -
 .../KllDoublesSketchIsEstimationMode.scala         | 82 ++++++++++++++++++++++
 .../spark/sql/datasketches/kll/functions.scala     | 34 ++++++++-
 6 files changed, 290 insertions(+), 5 deletions(-)

diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/KllFunctionRegistry.scala
 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/KllFunctionRegistry.scala
index 550e6c3..fa7e3d7 100644
--- 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/KllFunctionRegistry.scala
+++ 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/KllFunctionRegistry.scala
@@ -21,9 +21,14 @@ import 
org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
 import org.apache.spark.sql.catalyst.expressions.{ExpressionInfo}
 
 import org.apache.spark.sql.datasketches.common.DatasketchesFunctionRegistry
-import 
org.apache.spark.sql.datasketches.kll.aggregate.{KllDoublesSketchAggBuild, 
KllDoublesSketchAggMerge}
-import 
org.apache.spark.sql.datasketches.kll.expressions.{KllDoublesSketchGetMin, 
KllDoublesSketchGetMax, KllDoublesSketchGetPmf, KllDoublesSketchGetCdf}
-import org.apache.spark.sql.datasketches.common.DatasketchesFunctionRegistry
+import 
org.apache.spark.sql.datasketches.kll.aggregate.{KllDoublesSketchAggMerge, 
KllDoublesSketchAggBuild}
+import 
org.apache.spark.sql.datasketches.kll.expressions.{KllDoublesSketchGetMin,
+                                                          
KllDoublesSketchGetMax,
+                                                          
KllDoublesSketchGetPmf,
+                                                          
KllDoublesSketchGetCdf,
+                                                          
KllDoublesSketchGetNumRetained,
+                                                          KllDoublesSketchGetK,
+                                                          
KllDoublesSketchIsEstimationMode}
 
 object KllFunctionRegistry extends DatasketchesFunctionRegistry {
   override val expressions: Map[String, (ExpressionInfo, FunctionBuilder)] = 
Map(
@@ -33,5 +38,8 @@ object KllFunctionRegistry extends 
DatasketchesFunctionRegistry {
     expression[KllDoublesSketchGetMax]("kll_sketch_double_get_max"),
     expression[KllDoublesSketchGetPmf]("kll_sketch_double_get_pmf"),
     expression[KllDoublesSketchGetCdf]("kll_sketch_double_get_cdf"),
+    expression[KllDoublesSketchGetK]("kll_sketch_double_get_k"),
+    
expression[KllDoublesSketchGetNumRetained]("kll_sketch_double_get_num_retained"),
+    
expression[KllDoublesSketchIsEstimationMode]("kll_sketch_double_is_estimation_mode"),
   )
 }
diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetK.scala
 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetK.scala
new file mode 100644
index 0000000..a944cf2
--- /dev/null
+++ 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetK.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.datasketches.kll.expressions
+
+import org.apache.spark.sql.catalyst.expressions.{Expression,
+                                                  ExpressionDescription,
+                                                  UnaryExpression,
+                                                  ExpectsInputTypes,
+                                                  NullIntolerant}
+import org.apache.spark.sql.types.{AbstractDataType, DataType, DoubleType}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeBlock, 
CodegenContext, ExprCode}
+import org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType
+import org.apache.datasketches.kll.KllDoublesSketch
+import org.apache.datasketches.memory.Memory
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the configured size-accuracy parameter k given the 
binary representation
+    of a Datasketches KllDoublesSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(kll_sketch_agg(col, 200)) FROM VALUES (1.0), (2.0), 
(3.0) tab(col);
+       200
+  """
+  //group = "misc_funcs",
+)
+case class KllDoublesSketchGetK(sketchExpr: Expression)
+ extends UnaryExpression
+ with ExpectsInputTypes
+ with NullIntolerant {
+
+  override def child: Expression = sketchExpr
+
+  override protected def withNewChildInternal(newChild: Expression): 
KllDoublesSketchGetK = {
+    copy(sketchExpr = newChild)
+  }
+
+  override def prettyName: String = "kll_sketch_double_get_k"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(KllDoublesSketchType)
+
+  override def dataType: DataType = DoubleType
+
+  override def nullSafeEval(input: Any): Any = {
+    val bytes = input.asInstanceOf[Array[Byte]]
+    val sketch = KllDoublesSketch.wrap(Memory.wrap(bytes))
+    sketch.getK()
+  }
+
+  override protected def nullSafeCodeGen(ctx: CodegenContext, ev: ExprCode, f: 
String => String): ExprCode = {
+    val sketchEval = child.genCode(ctx)
+    val sketch = ctx.freshName("sketch")
+
+    val code =
+      s"""
+         |${sketchEval.code}
+         |final org.apache.datasketches.kll.KllDoublesSketch $sketch = 
org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType.wrap(${sketchEval.value});
+         |final int ${ev.value} = $sketch.getK();
+         |final boolean ${ev.isNull} = ${sketchEval.isNull};
+       """.stripMargin
+    ev.copy(code = CodeBlock(Seq(code), Seq.empty))
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    nullSafeCodeGen(ctx, ev, c => s"($c)")
+  }
+}
diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetNumRetained.scala
 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetNumRetained.scala
new file mode 100644
index 0000000..57df5e5
--- /dev/null
+++ 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetNumRetained.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.datasketches.kll.expressions
+
+import org.apache.spark.sql.catalyst.expressions.{Expression,
+                                                  ExpressionDescription,
+                                                  UnaryExpression,
+                                                  ExpectsInputTypes,
+                                                  NullIntolerant}
+import org.apache.spark.sql.types.{AbstractDataType, DataType, DoubleType}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeBlock, 
CodegenContext, ExprCode}
+import org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType
+import org.apache.datasketches.kll.KllDoublesSketch
+import org.apache.datasketches.memory.Memory
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns the number of items retained by the sketch given 
the binary representation
+    of a Datasketches KllDoublesSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(kll_sketch_agg(col)) FROM VALUES (1.0), (2.0), (3.0) 
tab(col);
+       3
+  """
+  //group = "misc_funcs",
+)
+case class KllDoublesSketchGetNumRetained(sketchExpr: Expression)
+ extends UnaryExpression
+ with ExpectsInputTypes
+ with NullIntolerant {
+
+  override def child: Expression = sketchExpr
+
+  override protected def withNewChildInternal(newChild: Expression): 
KllDoublesSketchGetNumRetained = {
+    copy(sketchExpr = newChild)
+  }
+
+  override def prettyName: String = "kll_sketch_double_get_num_retained"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(KllDoublesSketchType)
+
+  override def dataType: DataType = DoubleType
+
+  override def nullSafeEval(input: Any): Any = {
+    val bytes = input.asInstanceOf[Array[Byte]]
+    val sketch = KllDoublesSketch.wrap(Memory.wrap(bytes))
+    sketch.getNumRetained()
+  }
+
+  override protected def nullSafeCodeGen(ctx: CodegenContext, ev: ExprCode, f: 
String => String): ExprCode = {
+    val sketchEval = child.genCode(ctx)
+    val sketch = ctx.freshName("sketch")
+
+    val code =
+      s"""
+         |${sketchEval.code}
+         |final org.apache.datasketches.kll.KllDoublesSketch $sketch = 
org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType.wrap(${sketchEval.value});
+         |final int ${ev.value} = $sketch.getNumRetained();
+         |final boolean ${ev.isNull} = ${sketchEval.isNull};
+       """.stripMargin
+    ev.copy(code = CodeBlock(Seq(code), Seq.empty))
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    nullSafeCodeGen(ctx, ev, c => s"($c)")
+  }
+}
diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetPmfCdf.scala
 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetPmfCdf.scala
index 60d43ff..a041365 100644
--- 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetPmfCdf.scala
+++ 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchGetPmfCdf.scala
@@ -211,7 +211,6 @@ case class KllDoublesSketchGetPmfCdf(sketchExpr: Expression,
 // getRank(quantile, QuantileSearchCriteria)
 // getRanks(quantile[]), QuantileSearchCriteria)
 // getNormalizedRankError(bool isPmf)
-// isEstimationMode()
 // toString(bool, bool) -- already part of the wrapper
 // getK() ?
 // getNumRetained() ?
diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchIsEstimationMode.scala
 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchIsEstimationMode.scala
new file mode 100644
index 0000000..ee66824
--- /dev/null
+++ 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/expressions/KllDoublesSketchIsEstimationMode.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.datasketches.kll.expressions
+
+import org.apache.spark.sql.catalyst.expressions.{Expression,
+                                                  ExpressionDescription,
+                                                  UnaryExpression,
+                                                  ExpectsInputTypes,
+                                                  NullIntolerant}
+import org.apache.spark.sql.types.{AbstractDataType, DataType, DoubleType}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeBlock, 
CodegenContext, ExprCode}
+import org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType
+import org.apache.datasketches.kll.KllDoublesSketch
+import org.apache.datasketches.memory.Memory
+
+@ExpressionDescription(
+  usage = """
+    _FUNC_(expr) - Returns true if the sketch is in estimation mode, otherwise 
false given
+     the binary representation of a Datasketches KllDoublesSketch. """,
+  examples = """
+    Examples:
+      > SELECT _FUNC_(kll_sketch_agg(col, 200)) FROM VALUES (1.0), (2.0), 
(3.0) tab(col);
+       false
+  """
+  //group = "misc_funcs",
+)
+case class KllDoublesSketchIsEstimationMode(sketchExpr: Expression)
+ extends UnaryExpression
+ with ExpectsInputTypes
+ with NullIntolerant {
+
+  override def child: Expression = sketchExpr
+
+  override protected def withNewChildInternal(newChild: Expression): 
KllDoublesSketchIsEstimationMode = {
+    copy(sketchExpr = newChild)
+  }
+
+  override def prettyName: String = "kll_sketch_double_is_estimation_mode"
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(KllDoublesSketchType)
+
+  override def dataType: DataType = DoubleType
+
+  override def nullSafeEval(input: Any): Any = {
+    val bytes = input.asInstanceOf[Array[Byte]]
+    val sketch = KllDoublesSketch.wrap(Memory.wrap(bytes))
+    sketch.isEstimationMode()
+  }
+
+  override protected def nullSafeCodeGen(ctx: CodegenContext, ev: ExprCode, f: 
String => String): ExprCode = {
+    val sketchEval = child.genCode(ctx)
+    val sketch = ctx.freshName("sketch")
+
+    val code =
+      s"""
+         |${sketchEval.code}
+         |final org.apache.datasketches.kll.KllDoublesSketch $sketch = 
org.apache.spark.sql.datasketches.kll.types.KllDoublesSketchType.wrap(${sketchEval.value});
+         |final boolen ${ev.value} = $sketch.isEstimationMode();
+         |final boolean ${ev.isNull} = ${sketchEval.isNull};
+       """.stripMargin
+    ev.copy(code = CodeBlock(Seq(code), Seq.empty))
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    nullSafeCodeGen(ctx, ev, c => s"($c)")
+  }
+}
diff --git 
a/src/main/scala/org/apache/spark/sql/datasketches/kll/functions.scala 
b/src/main/scala/org/apache/spark/sql/datasketches/kll/functions.scala
index fe48c1c..9bebe3c 100644
--- a/src/main/scala/org/apache/spark/sql/datasketches/kll/functions.scala
+++ b/src/main/scala/org/apache/spark/sql/datasketches/kll/functions.scala
@@ -24,7 +24,12 @@ import org.apache.spark.sql.types.{ArrayType, BooleanType, 
DoubleType}
 
 import org.apache.spark.sql.datasketches.common.DatasketchesScalaFunctionBase
 import 
org.apache.spark.sql.datasketches.kll.aggregate.{KllDoublesSketchAggMerge, 
KllDoublesSketchAggBuild}
-import 
org.apache.spark.sql.datasketches.kll.expressions.{KllDoublesSketchGetMin, 
KllDoublesSketchGetMax, KllDoublesSketchGetPmfCdf}
+import 
org.apache.spark.sql.datasketches.kll.expressions.{KllDoublesSketchGetMin,
+                                                          
KllDoublesSketchGetMax,
+                                                          
KllDoublesSketchGetPmfCdf,
+                                                          
KllDoublesSketchGetNumRetained,
+                                                          KllDoublesSketchGetK,
+                                                          
KllDoublesSketchIsEstimationMode}
 
 object functions extends DatasketchesScalaFunctionBase {
 
@@ -70,6 +75,33 @@ object functions extends DatasketchesScalaFunctionBase {
     kll_sketch_double_agg_merge(Column(columnName), lit(k))
   }
 
+  // get k
+  def kll_sketch_double_get_k(expr: Column): Column = withExpr {
+    new KllDoublesSketchGetK(expr.expr)
+  }
+
+  def kll_sketch_double_get_k(columnName: String): Column = {
+    kll_sketch_double_get_k(Column(columnName))
+  }
+
+  // get num retained
+  def kll_sketch_double_get_num_retained(expr: Column): Column = withExpr {
+    new KllDoublesSketchGetNumRetained(expr.expr)
+  }
+
+  def kll_sketch_double_get_num_retained(columnName: String): Column = {
+    kll_sketch_double_get_num_retained(Column(columnName))
+  }
+
+  // is estimation mode
+  def kll_sketch_double_is_estimation_mode(expr: Column): Column = withExpr {
+    new KllDoublesSketchIsEstimationMode(expr.expr)
+  }
+
+  def kll_sketch_double_is_estimation_mode(columnName: String): Column = {
+    kll_sketch_double_is_estimation_mode(Column(columnName))
+  }
+
   // get min
   def kll_sketch_double_get_min(expr: Column): Column = withExpr {
     new KllDoublesSketchGetMin(expr.expr)


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to