You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "dario-liberman (via GitHub)" <gi...@apache.org> on 2023/07/24 19:05:13 UTC

[GitHub] [pinot] dario-liberman commented on a diff in pull request #11092: Funnel Count - Multiple Strategies (no partitioning requisites)

dario-liberman commented on code in PR #11092:
URL: https://github.com/apache/pinot/pull/11092#discussion_r1272643837


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/FunnelCountAggregationFunction.java:
##########
@@ -35,46 +50,107 @@
 import org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
 import org.apache.pinot.segment.spi.AggregationFunctionType;
 import org.apache.pinot.segment.spi.index.reader.Dictionary;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.roaringbitmap.PeekableIntIterator;
 import org.roaringbitmap.RoaringBitmap;
 
 
 /**
- * The {@code FunnelCountAggregationFunction} calculates the number of step conversions for a given partition column and
- * a list of boolean expressions.
- * <p>IMPORTANT: This function relies on the partition column being partitioned for each segment, where there are no
- * common values across different segments.
- * <p>This function calculates the exact number of step matches per partition key within the segment, then sums up the
- * results from different segments.
+ * The {@code FunnelCountAggregationFunction} calculates the number of conversions for a given correlation column and
+ * a list of steps as boolean expressions.
  *
  * Example:
  *   SELECT
  *    dateTrunc('day', timestamp) AS ts,
  *    FUNNEL_COUNT(
  *      STEPS(url = '/addToCart', url = '/checkout', url = '/orderConfirmation'),
- *      CORRELATED_BY(user)
+ *      CORRELATED_BY(user_id)
  *    ) as step_counts
  *    FROM user_log
  *    WHERE url in ('/addToCart', '/checkout', '/orderConfirmation')
  *    GROUP BY 1
+ *
+ *  Counting strategies can be controlled via optional SETTINGS options, for example:
+ *
+ *  FUNNEL_COUNT(
+ *    STEPS(url = '/addToCart', url = '/checkout', url = '/orderConfirmation'),
+ *    CORRELATED_BY(user_id),
+ *    SETTINGS('theta_sketch','nominalEntries=4096')
+ *  )
+ *
+ *  There are 5 strategies available, mirroring the corresponding distinct count implementations as per below.
+ *  <p><ul>
+ *  <li>'set': See DISTINCTCOUNT at {@link DistinctCountAggregationFunction}
+ *  <li>'bitmap' (default): See DISTINCTCOUNTBITMAP at {@link DistinctCountBitmapAggregationFunction}
+ *  <li>'theta_sketch': See DISTINCTCOUNTTHETASKETCH at {@link DistinctCountThetaSketchAggregationFunction}
+ *  <li>'partitioned': See SEGMENTPARTITIONEDDISTINCTCOUNT {@link SegmentPartitionedDistinctCountAggregationFunction}
+ *  <li>'sorted': sorted counts per segment then sums up. Only availabe in combination with 'partitioned'.
+ *  <li>'nominalEntries=4096': theta sketch configuration, default is 4096.
+ *  </ul><p>
  */
-public class FunnelCountAggregationFunction implements AggregationFunction<List<Long>, LongArrayList> {
+public class FunnelCountAggregationFunction implements AggregationFunction<Object, LongArrayList> {
+  private static final Sketch EMPTY_SKETCH = new UpdateSketchBuilder().build().compact();
   final List<ExpressionContext> _expressions;
   final List<ExpressionContext> _stepExpressions;
   final List<ExpressionContext> _correlateByExpressions;
   final ExpressionContext _primaryCorrelationCol;
   final int _numSteps;
-
-  final SegmentAggregationStrategy<?, List<Long>> _sortedAggregationStrategy;
-  final SegmentAggregationStrategy<?, List<Long>> _bitmapAggregationStrategy;
+  final int _nominalEntries;
+  final boolean _partitionSetting;
+  final boolean _sortingSetting;
+  final boolean _thetaSketchSetting;
+  final boolean _setSetting;
+
+  final AggregationStrategy _thetaSketchAggregationStrategy;
+  final AggregationStrategy _bitmapAggregationStrategy;
+  final AggregationStrategy _sortedAggregationStrategy;
+
+  final MergeStrategy _thetaSketchMergeStrategy;
+  final MergeStrategy _setMergeStrategy;
+  final MergeStrategy _bitmapMergeStrategy;
+  final MergeStrategy _partitionedMergeStrategy;
+
+  final ResultExtractionStrategy _thetaSketchResultExtractionStrategy;
+  final ResultExtractionStrategy _setResultExtractionStrategy;
+  final ResultExtractionStrategy _bitmapResultExtractionStrategy;
+  final ResultExtractionStrategy _sortedPartitionedResultExtractionStrategy;
+  final ResultExtractionStrategy _bitmapPartitionedResultExtractionStrategy;
+  final ResultExtractionStrategy _thetaSketchPartitionedResultExtractionStrategy;
 
   public FunnelCountAggregationFunction(List<ExpressionContext> expressions) {
     _expressions = expressions;
+    Option.validate(expressions);
     _correlateByExpressions = Option.CORRELATE_BY.getInputExpressions(expressions);
     _primaryCorrelationCol = Option.CORRELATE_BY.getFirstInputExpression(expressions);
     _stepExpressions = Option.STEPS.getInputExpressions(expressions);
     _numSteps = _stepExpressions.size();
-    _sortedAggregationStrategy = new SortedAggregationStrategy();
+
+    final List<String> settings = Option.SETTINGS.getLiterals(expressions);
+    Setting.validate(settings);

Review Comment:
   Some combinations are valid. For example, one could ask for `partitioned` together with `sorted` or together with `tetha_sketch`.
   But we could indeed check for invalid combinations instead of just prioritising them.



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org