You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/05/06 18:42:41 UTC

[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5339: Add api in AggregationFunction to get its compiled input expressions.

Jackie-Jiang commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421006375



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/plan/TransformPlanNode.java
##########
@@ -43,76 +39,70 @@
 
   private final String _segmentName;
   private final ProjectionPlanNode _projectionPlanNode;
-  private final Set<String> _projectionColumns = new HashSet<>();
-  private final Set<TransformExpressionTree> _expressions = new HashSet<>();
+  private final Set<TransformExpressionTree> _expressions;
   private int _maxDocPerNextCall = DocIdSetPlanNode.MAX_DOC_PER_CALL;
 
-  public TransformPlanNode(IndexSegment indexSegment, BrokerRequest brokerRequest) {
+  public TransformPlanNode(IndexSegment indexSegment, BrokerRequest brokerRequest,
+      Set<TransformExpressionTree> expressionsToPlan) {
     _segmentName = indexSegment.getSegmentName();
-    extractColumnsAndTransforms(brokerRequest, indexSegment);
-    _projectionPlanNode = new ProjectionPlanNode(indexSegment, _projectionColumns,
+
+    setMaxDocsForSelection(brokerRequest);
+    Set<String> projectionColumns = new HashSet<>();
+    extractProjectionColumns(expressionsToPlan, projectionColumns);
+
+    _expressions = expressionsToPlan;
+    _projectionPlanNode = new ProjectionPlanNode(indexSegment, projectionColumns,
         new DocIdSetPlanNode(indexSegment, brokerRequest, _maxDocPerNextCall));
   }
 
+  private void extractProjectionColumns(Set<TransformExpressionTree> expressionsToPlan, Set<String> projectionColumns) {
+    for (TransformExpressionTree expression : expressionsToPlan) {
+      extractProjectionColumns(expression, projectionColumns);
+    }
+  }
+
+  private void extractProjectionColumns(TransformExpressionTree expression, Set<String> projectionColumns) {
+    TransformExpressionTree.ExpressionType expressionType = expression.getExpressionType();
+    switch (expressionType) {
+      case FUNCTION:
+        for (TransformExpressionTree child : expression.getChildren()) {
+          extractProjectionColumns(child, projectionColumns);
+        }
+        break;
+
+      case IDENTIFIER:
+        projectionColumns.add(expression.getValue());
+        break;
+
+      case LITERAL:
+        // Do nothing.
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported expression type: " + expressionType);
+    }
+  }
+
   /**
-   * Helper method to extract projection columns and transform expressions from the given broker request.
+   * Helper method to set the max number of docs to return for selection queries
    */
-  private void extractColumnsAndTransforms(BrokerRequest brokerRequest, IndexSegment indexSegment) {
-    Set<String> columns = new HashSet<>();
-    if (brokerRequest.isSetAggregationsInfo()) {
-      // Extract aggregation expressions
-      for (AggregationInfo aggregationInfo : brokerRequest.getAggregationsInfo()) {
-        if (aggregationInfo.getAggregationType().equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) {
-          // 'DISTINCT(col1, col2 ...)' is modeled as one single aggregation function
-          List<String> distinctColumns = AggregationFunctionUtils.getAggregationExpressions(aggregationInfo);
-          columns.addAll(distinctColumns);
-        } else if (!aggregationInfo.getAggregationType().equalsIgnoreCase(AggregationFunctionType.COUNT.getName())) {
-          columns.addAll(AggregationFunctionUtils.getAggregationExpressions(aggregationInfo));
-        }
-      }
-      // Extract group-by expressions
-      if (brokerRequest.isSetGroupBy()) {
-        columns.addAll(brokerRequest.getGroupBy().getExpressions());
-      }
-    } else {
+  private void setMaxDocsForSelection(BrokerRequest brokerRequest) {

Review comment:
       This logic should also be handle in the upper level (SelectionPlanNode) and passed to this class

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/plan/TransformPlanNode.java
##########
@@ -43,76 +39,70 @@
 
   private final String _segmentName;
   private final ProjectionPlanNode _projectionPlanNode;
-  private final Set<String> _projectionColumns = new HashSet<>();
-  private final Set<TransformExpressionTree> _expressions = new HashSet<>();
+  private final Set<TransformExpressionTree> _expressions;
   private int _maxDocPerNextCall = DocIdSetPlanNode.MAX_DOC_PER_CALL;
 
-  public TransformPlanNode(IndexSegment indexSegment, BrokerRequest brokerRequest) {
+  public TransformPlanNode(IndexSegment indexSegment, BrokerRequest brokerRequest,
+      Set<TransformExpressionTree> expressionsToPlan) {

Review comment:
       (nit) `expressionsToPlan` -> `expressions`?
   
   Also pass `maxDocsPerBlock` from upper level?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunction.java
##########
@@ -49,6 +51,13 @@
    */
   String getResultColumnName();
 
+  /**
+   * Returns a list of input expressions needed for performing aggregation.
+   *
+   * @return List of input expressions

Review comment:
       (nit) We don't have return argument documented in this file, maybe keep it consistent?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java
##########
@@ -182,4 +187,37 @@ public static String concatArgs(List<String> arguments) {
     return (arguments.size() > 1) ? String.join(CompilerConstants.AGGREGATION_FUNCTION_ARG_SEPARATOR, arguments)
         : arguments.get(0);
   }
+
+  /**
+   * Compiles and returns all transform expressions required for computing the aggregation, group-by
+   * and order-by
+   *
+   * @param brokerRequest Broker Request
+   * @param functionContexts Aggregation Function contexts
+   * @return Set of compiled expressions in the aggregation, group-by and order-by clauses
+   */
+  public static Set<TransformExpressionTree> collectExpressionsToTransform(BrokerRequest brokerRequest,
+      AggregationFunctionContext[] functionContexts) {
+
+    Set<TransformExpressionTree> expressionTrees = new LinkedHashSet<>();
+    for (AggregationFunctionContext functionContext : functionContexts) {
+      AggregationFunction function = functionContext.getAggregationFunction();
+
+      // Count is always treated as count(*) and does not need any columns to be projected.
+      if (!function.getType().equals(AggregationFunctionType.COUNT)) {

Review comment:
       No need to specialize this. Just make CountAggregationFunction return empty list

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/CountAggregationFunction.java
##########
@@ -57,6 +59,11 @@ public String getResultColumnName() {
     return AggregationFunctionType.COUNT.getName().toLowerCase() + "(*)";
   }
 
+  @Override
+  public List<TransformExpressionTree> getInputExpressions() {
+    return null;

Review comment:
       ```suggestion
       return Collections.emptyList();
   ```




----------------------------------------------------------------
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.

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