Jackie-Jiang commented on code in PR #14941:
URL: https://github.com/apache/pinot/pull/14941#discussion_r2017624085


##########
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/funnel/window/FunnelEventsFunctionEvalAggregationFunction.java:
##########
@@ -0,0 +1,502 @@
+/**
+ * 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.query.aggregation.function.funnel.window;
+
+import com.google.common.base.Preconditions;
+import it.unimi.dsi.fastutil.objects.ObjectArrayList;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.request.context.ExpressionContext;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
+import org.apache.pinot.core.query.aggregation.function.funnel.FunnelStepEvent;
+import 
org.apache.pinot.core.query.aggregation.function.funnel.FunnelStepEventWithExtraFields;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import 
org.apache.pinot.core.query.aggregation.groupby.ObjectGroupByResultHolder;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+
+
+public class FunnelEventsFunctionEvalAggregationFunction
+    implements 
AggregationFunction<PriorityQueue<FunnelStepEventWithExtraFields>, 
ObjectArrayList<String>> {
+  protected final ExpressionContext _timestampExpression;
+  protected final long _windowSize;
+  protected final List<ExpressionContext> _stepExpressions;
+  protected final FunnelBaseAggregationFunction.FunnelModes _modes = new 
FunnelBaseAggregationFunction.FunnelModes();
+  protected final int _numSteps;
+  protected final int _numExtraFields;
+  protected final List<ExpressionContext> _extraExpressions;
+  protected long _maxStepDuration = 0L;
+
+  public FunnelEventsFunctionEvalAggregationFunction(List<ExpressionContext> 
arguments) {
+    int numArguments = arguments.size();
+    Preconditions.checkArgument(numArguments > 3,
+        "FUNNEL_EVENTS_FUNCTION_EVAL expects >= 4 arguments, got: %s. The 
function can be used as "
+            + getType().getName() + "(timestampExpression, windowSize, 
numberSteps, stepExpression, "
+            + "[stepExpression, ..], [mode, [mode, ... ]])",
+        numArguments);
+    _timestampExpression = arguments.get(0);
+    _windowSize = arguments.get(1).getLiteral().getLongValue();
+    Preconditions.checkArgument(_windowSize > 0, "Window size must be > 0");
+    _numSteps = arguments.get(2).getLiteral().getIntValue();
+    Preconditions.checkArgument(numArguments >= 3 + _numSteps,
+        "FUNNEL_EVENTS_FUNCTION_EVAL expects >= " + (3 + _numSteps)
+            + " arguments, got: %s. The function can be used as "
+            + getType().getName() + "(timestampExpression, windowSize, 
numberSteps, stepExpression, "
+            + "[stepExpression, ..], [extraArgument/mode, [extraArgument/mode, 
... ]])",
+        numArguments);
+    _stepExpressions = arguments.subList(3, 3 + _numSteps);
+    _numExtraFields = arguments.get(3 + _numSteps).getLiteral().getIntValue();
+    Preconditions.checkArgument(numArguments >= 4 + _numSteps + 
_numExtraFields,
+        "FUNNEL_EVENTS_FUNCTION_EVAL expects >= " + (4 + _numSteps + 
_numExtraFields)
+            + " arguments, got: %s. The function can be used as "
+            + getType().getName() + "(timestampExpression, windowSize, 
numberSteps, stepExpression, "
+            + "[stepExpression, ..], [extraArgument/mode, [extraArgument/mode, 
... ]])",
+        numArguments);
+    _extraExpressions = arguments.subList(4 + _numSteps, 4 + _numSteps + 
_numExtraFields);
+
+    for (int i = 4 + _numSteps + _numExtraFields; i < numArguments; i++) {
+      String extraArgument = 
arguments.get(i).getLiteral().getStringValue().toUpperCase();
+      String[] parsedExtraArguments = extraArgument.split("=");
+      if (parsedExtraArguments.length == 2) {
+        String key = parsedExtraArguments[0].toUpperCase();
+        switch (key) {
+          case FunnelBaseAggregationFunction.FunnelConfigs.MAX_STEP_DURATION:
+            _maxStepDuration = Long.parseLong(parsedExtraArguments[1]);
+            Preconditions.checkArgument(_maxStepDuration > 0, "MaxStepDuration 
must be > 0");
+            break;
+          case FunnelBaseAggregationFunction.FunnelConfigs.MODE:
+            for (String modeStr : parsedExtraArguments[1].split(",")) {
+              
_modes.add(FunnelBaseAggregationFunction.Mode.valueOf(modeStr.trim()));
+            }
+            break;
+          default:
+            throw new IllegalArgumentException("Unrecognized arguments: " + 
extraArgument);
+        }
+        continue;
+      }
+      try {
+        _modes.add(FunnelBaseAggregationFunction.Mode.valueOf(extraArgument));
+      } catch (Exception e) {
+        throw new RuntimeException("Unrecognized extra argument for funnel 
function: " + extraArgument, e);
+      }
+    }
+  }
+
+  @Override
+  public String getResultColumnName() {
+    return
+        String.format("%s(%d)(%s,%s,%s)", getType().getName(), _windowSize, 
_timestampExpression.toString(),
+            
_stepExpressions.stream().map(ExpressionContext::toString).collect(Collectors.joining(",")),
+            (_numExtraFields > 0 ? ", " + 
_extraExpressions.stream().map(ExpressionContext::toString)
+                .collect(Collectors.joining(",")) : "")
+        );
+  }
+
+  @Override
+  public List<ExpressionContext> getInputExpressions() {
+    List<ExpressionContext> inputs = new ArrayList<>(1 + _numSteps + 
_numExtraFields);
+    inputs.add(_timestampExpression);
+    inputs.addAll(_stepExpressions);
+    inputs.addAll(_extraExpressions);
+    return inputs;
+  }
+
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+    return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public GroupByResultHolder createGroupByResultHolder(int initialCapacity, 
int maxCapacity) {
+    return new ObjectGroupByResultHolder(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder 
aggregationResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    long[] timestampBlock = 
blockValSetMap.get(_timestampExpression).getLongValuesSV();
+    List<int[]> stepBlocks = new ArrayList<>(_numSteps);
+    for (ExpressionContext stepExpression : _stepExpressions) {
+      stepBlocks.add(blockValSetMap.get(stepExpression).getIntValuesSV());
+    }
+    PriorityQueue<FunnelStepEventWithExtraFields> stepEvents = 
aggregationResultHolder.getResult();
+    if (stepEvents == null) {
+      stepEvents = new PriorityQueue<>();
+      aggregationResultHolder.setValue(stepEvents);
+    }
+    List<Object> extraFieldsBlocks = getExtraFieldsBlocks(blockValSetMap);
+    for (int i = 0; i < length; i++) {
+      boolean stepFound = false;
+      for (int j = 0; j < _numSteps; j++) {
+        if (stepBlocks.get(j)[i] == 1) {
+          List<Object> extraFields = extractExtraFields(extraFieldsBlocks, i);
+          stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], j), extraFields));
+          stepFound = true;
+          break;
+        }
+      }
+      // If the mode is KEEP_ALL and no step is found, add a dummy step event 
with step -1
+      if (_modes.hasKeepAll() && !stepFound) {
+        List<Object> extraFields = extractExtraFields(extraFieldsBlocks, i);
+        stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], -1), extraFields));
+      }
+    }
+  }
+
+  private List<Object> getExtraFieldsBlocks(Map<ExpressionContext, 
BlockValSet> blockValSetMap) {
+    List<Object> extraFieldsBlocks = new ArrayList<>(_numExtraFields);
+    for (ExpressionContext extraExpression : _extraExpressions) {
+      BlockValSet blockValSet = blockValSetMap.get(extraExpression);
+      switch (blockValSet.getValueType()) {
+        case INT:
+          extraFieldsBlocks.add(blockValSet.getIntValuesSV());
+          break;
+        case LONG:
+        case TIMESTAMP:
+          extraFieldsBlocks.add(blockValSet.getLongValuesSV());
+          break;
+        case FLOAT:
+          extraFieldsBlocks.add(blockValSet.getFloatValuesSV());
+          break;
+        case DOUBLE:
+          extraFieldsBlocks.add(blockValSet.getDoubleValuesSV());
+          break;
+        case STRING:
+          extraFieldsBlocks.add(blockValSet.getStringValuesSV());
+          break;
+        default:
+          throw new IllegalArgumentException("Unsupported data type for extra 
field: " + extraExpression + " - "
+              + blockValSet.getValueType());
+      }
+    }
+    return extraFieldsBlocks;
+  }
+
+  private List<Object> extractExtraFields(List<Object> extraFieldsBlocks, int 
i) {
+    List<Object> extraFields = new ArrayList<>(_numExtraFields);
+    for (Object extraFieldsBlock : extraFieldsBlocks) {
+      switch (extraFieldsBlock.getClass().getComponentType().getSimpleName()) {
+        case "int":
+          extraFields.add(((int[]) extraFieldsBlock)[i]);
+          break;
+        case "long":
+          extraFields.add(((long[]) extraFieldsBlock)[i]);
+          break;
+        case "float":
+          extraFields.add(((float[]) extraFieldsBlock)[i]);
+          break;
+        case "double":
+          extraFields.add(((double[]) extraFieldsBlock)[i]);
+          break;
+        case "String":
+          extraFields.add(((String[]) extraFieldsBlock)[i]);
+          break;
+        default:
+          throw new IllegalArgumentException(
+              "Unsupported data type for extra field: " + 
extraFieldsBlock.getClass().getComponentType()
+                  .getSimpleName());
+      }
+    }
+    return extraFields;
+  }
+
+  @Override
+  public void aggregateGroupBySV(int length, int[] groupKeyArray, 
GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    long[] timestampBlock = 
blockValSetMap.get(_timestampExpression).getLongValuesSV();
+    List<int[]> stepBlocks = new ArrayList<>(_numSteps);
+    for (ExpressionContext stepExpression : _stepExpressions) {
+      stepBlocks.add(blockValSetMap.get(stepExpression).getIntValuesSV());
+    }
+    List<Object> extraFieldsBlocks = getExtraFieldsBlocks(blockValSetMap);
+    for (int i = 0; i < length; i++) {
+      int groupKey = groupKeyArray[i];
+      boolean stepFound = false;
+      for (int j = 0; j < _numSteps; j++) {
+        if (stepBlocks.get(j)[i] == 1) {
+          PriorityQueue<FunnelStepEventWithExtraFields> stepEvents = 
getFunnelStepEvents(groupByResultHolder, groupKey);
+          List<Object> extraFields = extractExtraFields(extraFieldsBlocks, i);
+          stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], j), extraFields));
+          stepFound = true;
+          break;
+        }
+      }
+      // If the mode is KEEP_ALL and no step is found, add a dummy step event 
with step -1
+      if (_modes.hasKeepAll() && !stepFound) {
+        PriorityQueue<FunnelStepEventWithExtraFields> stepEvents = 
getFunnelStepEvents(groupByResultHolder, groupKey);
+        List<Object> extraFields = extractExtraFields(extraFieldsBlocks, i);
+        stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], -1), extraFields));
+      }
+    }
+  }
+
+  @Override
+  public void aggregateGroupByMV(int length, int[][] groupKeysArray, 
GroupByResultHolder groupByResultHolder,
+      Map<ExpressionContext, BlockValSet> blockValSetMap) {
+    long[] timestampBlock = 
blockValSetMap.get(_timestampExpression).getLongValuesSV();
+    List<int[]> stepBlocks = new ArrayList<>(_numSteps);
+    for (ExpressionContext stepExpression : _stepExpressions) {
+      stepBlocks.add(blockValSetMap.get(stepExpression).getIntValuesSV());
+    }
+    List<Object> extraFieldsBlocks = getExtraFieldsBlocks(blockValSetMap);
+    for (int i = 0; i < length; i++) {
+      int[] groupKeys = groupKeysArray[i];
+      boolean stepFound = false;
+      for (int j = 0; j < _numSteps; j++) {
+        if (stepBlocks.get(j)[i] == 1) {
+          for (int groupKey : groupKeys) {
+            PriorityQueue<FunnelStepEventWithExtraFields> stepEvents =
+                getFunnelStepEvents(groupByResultHolder, groupKey);
+            List<Object> extraFields = extractExtraFields(extraFieldsBlocks, 
i);
+            stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], j), extraFields));
+          }
+          stepFound = true;
+          break;
+        }
+      }
+      // If the mode is KEEP_ALL and no step is found, add a dummy step event 
with step -1
+      if (_modes.hasKeepAll() && !stepFound) {
+        for (int groupKey : groupKeys) {
+          PriorityQueue<FunnelStepEventWithExtraFields> stepEvents = 
getFunnelStepEvents(groupByResultHolder, groupKey);
+          List<Object> extraFields = extractExtraFields(extraFieldsBlocks, i);
+          stepEvents.add(new FunnelStepEventWithExtraFields(new 
FunnelStepEvent(timestampBlock[i], -1), extraFields));
+        }
+      }
+    }
+  }
+
+  private static PriorityQueue<FunnelStepEventWithExtraFields> 
getFunnelStepEvents(
+      GroupByResultHolder groupByResultHolder,
+      int groupKey) {
+    PriorityQueue<FunnelStepEventWithExtraFields> stepEvents = 
groupByResultHolder.getResult(groupKey);
+    if (stepEvents == null) {
+      stepEvents = new PriorityQueue<>();
+      groupByResultHolder.setValueForKey(groupKey, stepEvents);
+    }
+    return stepEvents;
+  }
+
+  @Override
+  public PriorityQueue<FunnelStepEventWithExtraFields> 
extractAggregationResult(
+      AggregationResultHolder aggregationResultHolder) {
+    return aggregationResultHolder.getResult();
+  }
+
+  @Override
+  public PriorityQueue<FunnelStepEventWithExtraFields> 
extractGroupByResult(GroupByResultHolder groupByResultHolder,
+      int groupKey) {
+    return groupByResultHolder.getResult(groupKey);
+  }
+
+  @Override
+  public PriorityQueue<FunnelStepEventWithExtraFields> merge(
+      PriorityQueue<FunnelStepEventWithExtraFields> intermediateResult1,
+      PriorityQueue<FunnelStepEventWithExtraFields> intermediateResult2) {
+    if (intermediateResult1 == null) {
+      return intermediateResult2;
+    }
+    if (intermediateResult2 == null) {
+      return intermediateResult1;
+    }
+    intermediateResult1.addAll(intermediateResult2);
+    return intermediateResult1;
+  }
+
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+    return DataSchema.ColumnDataType.STRING_ARRAY;

Review Comment:
   This should be `OBJECT`, and you may override the ser/de method. You 
shouldn't need to modify other classes for ser/de purpose



-- 
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-unsubscr...@pinot.apache.org

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


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

Reply via email to