walterddr commented on code in PR #10347:
URL: https://github.com/apache/pinot/pull/10347#discussion_r1119014136


##########
pinot-core/src/main/java/org/apache/pinot/core/function/scalar/SketchFunctions.java:
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.pinot.core.function.scalar;
+
+import com.clearspring.analytics.stream.cardinality.HyperLogLog;
+import java.math.BigDecimal;
+import org.apache.datasketches.theta.Sketches;
+import org.apache.datasketches.theta.UpdateSketch;
+import org.apache.pinot.core.common.ObjectSerDeUtils;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+/**
+ * Inbuilt Sketch Transformation Functions
+ * The functions can be used as UDFs in Query when added in the 
FunctionRegistry.
+ * @ScalarFunction annotation is used with each method for the registration
+ *
+ * These are intended to be used during ingestion to create sketches from raw 
data, which can be rolled up later.
+ *
+ * Note this is defined in pinot-core rather than pinot-common because 
pinot-core has dependencies on
+ * datasketches/clearspring analytics.
+ *
+ * Example usage:
+ * {
+ *   "transformConfigs": [
+ *     {
+ *       "columnName": "players",
+ *       "transformFunction": "DistinctCountRawThetaSketch(playerID)"
+ *     },
+ *     {
+ *       "columnName": "names",
+ *       "transformFunction": "DistinctCountRawHLL(playerName)"
+ *     }
+ *   ]
+ * }
+ */
+public class SketchFunctions {
+  private SketchFunctions() {
+  }
+
+  /**
+   * Create a Theta Sketch containing the input
+   *
+   * @param input an Object we want to insert into the sketch, may be null to 
return an empty sketch
+   * @return serialized theta sketch as bytes
+   */
+  @ScalarFunction(nullableParameters = true)
+  public static byte[] distinctCountRawThetaSketch(Object input) {
+    // TODO make nominal entries configurable
+    UpdateSketch sketch =
+        
Sketches.updateSketchBuilder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES)

Review Comment:
   btw, is this entry event designed to be configurable? if we are always only 
adding 1 value to the sketch (same as HLL)
   
   - this means we pre-determine the storage/accuracy trade-off during 
ingestion yes? 
   - what happen if some time down the line we changed the nominal entries 
value and some previously ingested data was on a different configuration? will 
the merge/rollup still work?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to