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 05:27:46 UTC

[GitHub] [incubator-pinot] mayankshriv opened a new pull request #5339: Add api in AggregationFunction to return the its compiled input expressions.

mayankshriv opened a new pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339


   With aggregation functions now taking multiple agruments, only the functions themselves
   have the knowledge on how to interpret these arguments. This poses a problem for the planning
   phase on what columns need to be projected and what expressions need to be computed.
   With this change, AggregationFunction's are now responsible for providing what inputs they need.
   
   1. Added a new api in AggregationFunction interface `getInputExpressions()`, that returns a list
      of compiled TransformExpressionTrees that the aggregation function needs as input to compute.
   
   2. Cleaned up the chaining data dependency during planning phase. Before this PR, all planning nodes
      receive the BrokerRequest (and pass to their child plan node) to extract out all information needed.
      With this change:
      - Aggregation plan nodes only specify the expression trees they need from Transform plan nodes, and
        Transform plan nodes use that to specify what columns they need from projection plan nodes.
   
   TODO: Ideally we should completely eliminate passing of BrokerRequest throughout the chain plan nodes,
   and only pass minimal information instead. This change only does so for projection columns. A TODO here
   is to extend it to FilterPlanNode and deeper.


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


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

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421062037



##########
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:
       👍 




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


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

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421060222



##########
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:
       Ok, will remove. It does not enough value to go out of the norm I guess.




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


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

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421060734



##########
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:
       👍 




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421059479



##########
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:
       `expressions` was too generic and being used in too many places for different purposes, so I thought to call it `expressionsToPlan`.
   Replied on `maxDocsPerBlock` above.




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


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

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5339:
URL: https://github.com/apache/incubator-pinot/pull/5339#discussion_r421058069



##########
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:
       Yeah I thought so too. But upper levels can be aggregation as well, which will now have this field leaked. For now I tend to keep it here (as was the case before) until the full cleanup happens.




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