You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "somandal (via GitHub)" <gi...@apache.org> on 2023/02/15 04:29:18 UTC

[GitHub] [pinot] somandal opened a new pull request, #10286: [multistage] Initial (phase 1) Add query runtime support for empty OVER() and OVER(PARTITION BY) window functions

somandal opened a new pull request, #10286:
URL: https://github.com/apache/pinot/pull/10286

   This PR introduces the multi-stage runtime changes to support Phase 1 of Window Functions, specifically targeting empty `OVER()` and `OVER(PARTITION BY)`. Runtime support has been added for the following classes of window queries:
   
   - Empty OVER()
   - OVER(PARTITION BY)
   
   This PR also ads/fixes some bugs in the window function planning, specifically:
   
   - Create Empty `OVER()` and `OVER(ORDER BY)` as singleton stages which need a single node assignment (similar to global aggregation without group by and global sort)
   - Add some logic for `OVER(PARTITION BY key1 ORDER BY key1)` queries which get marked as `OVER(PARTITION BY key1)` to ensure that the `ORDER BY` key1's direction and null direction are the defaults, otherwise treat these as `OVER(PARTITION BY key1 ORDER BY key2)` type queries during planning.
   
   The window functions supported as part of Phase 1 are: SUM, AVG, MIN, MAX, and COUNT.
   
   This PR also adds a new JSON file for window function query runtime tests via the output mechanism (H2 doesn't support window functions and got parse errors when trying to use it). (I still need to add some more tests, working on them now)
   
   Design document: https://docs.google.com/document/d/13CmFm4djI09JKF_Xty5acoXxJoxC9CLXAsmikgzgtIs/edit?usp=sharing
   OSS issue: https://github.com/apache/pinot/issues/7213
   Window Function Planner PR: https://github.com/apache/pinot/pull/10228
   
   This PR does not include support for:
   
   - Execution engine changes for Phase 1 for OVER(ORDER BY) and OVER (PARTITION BY ORDER BY) (will be the next PR for this feature)
   - Custom frames
   - Other window functions related to rank and values
   - Multiple window groups (basically multiple OVER clauses with different PARTITION BY, ORDER BY and/or FRAME specifications, if these specifications are the same they get grouped into a single window group)
   
   The above will be part of future changes for window function support
   
   cc @siddharthteotia @walterddr @vvivekiyer @Jackie-Jiang 


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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122649991


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.

Review Comment:
   Not sure I follow this. What is the special casing here w.r.t single value v/s multi value ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122751054


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   The state that whether or not we can produce output block should be maintained and dictated by the piece of code that consumed the input block - `consumeInputBlocks`
   
   Generally speaking, whenever an operator consumes an input block, before exiting that function it should be able to determine whether or not it is ready to produce output block and accordingly change the local state which can then be leveraged in `getNextBlock`.
   
   



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122687450


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());

Review Comment:
   Will we throw compilation error if after merging this PR, someone attempts `RANK` or say `ROW_NUMBER`  or say value functions ?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110459576


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {

Review Comment:
   done



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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1112385721


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Users can have a query where they first perform windowing in a sub-query, and then later perform aggregation with some filtering. Taking a Uber themed example, say there's a table which has an entry for each Eats order `(order_id, price,  category=[chinese, indian, italian, etc.], food_item=[burger, pizza, etc.])`, a query may be to find how many categories are there such that the second highest order food_item per category has more than 10k orders in the last day.
   
   If there are no such food categories, ideally we should return a single row `[(0)]`, but if we don't return any block above the window operator we may end up returning no rows `[]`. (@walterddr can perhaps confirm this since he had fixed a related issue with empty response in agg operators)
   
   Though this is a very specific example, I think it might be worthwhile to keep an invariant that each Operator would process at least one block.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122664596


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {

Review Comment:
   (nit) is it possible to encapsulate the input into an POJO type object ? Might look cleaner / easy to read / review. 
   
   Not necessary for this PR though



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122724937


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);

Review Comment:
   I think 248 and 249 can be combined into a single statement to avoid looking up key twice in `partitionRows` ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122725337


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {

Review Comment:
   Suggest moving size() before loop



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1118116891


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Yup makes sense! I've changed this back to return an `EOS` since this doesn't apply to window functions and the `AggregateOperator` handles this case.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1119474659


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -20,6 +20,7 @@
 
 import com.clearspring.analytics.util.Preconditions;
 import java.util.ArrayList;
+import java.util.Collections;

Review Comment:
   Related to planner PR:
   Is there value is repeating the `validateFrameBounds()` checks here? We have a similar check in [PinotWindowExchangeNodeInsertRule.java](https://github.com/apache/pinot/pull/10286/files#diff-347310fedfd85bcf7bb0c4a5aac5f671760f6c0f3f4a65352648499aaa5eb0b0)



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,343 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {

Review Comment:
   In Phase1 (not specific to this PR), are we going to support queries where there are multiple window functions but one function has an ORDER BY and the other doesn't? I'm assuming it fails the criteria - "All order by keys should be on a single column" and Calcite will generate 2 window groups. Is that correct?
   
   `SELECT SUM(col1) OVER(PARTITION BY key1), SUM(col2) OVER(PARTITION BY key1 ORDER BY col1) from table t1`
   
   Either way, If we don't have a test for this case, can we please add them
   



##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   +1 to removing this because:
   1. We use isPartitionByOnlyQuery to decide if we have to use LogicalExchange vs LogicalSortExchange
   2. Based on my understanding, no sorting of rows within a partition is needed if the partition key and order by key are the same. 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,343 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include

Review Comment:
   We also support AVG by virtue of SUM and COUNT right?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123302555


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;

Review Comment:
   Great suggestion! I'll revisit these once I start work on frame support as I want to look into whether there is a recommendation on the frame limits or not. Will take this as an action item for frame support.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123666827


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   Sounds good. FYI @walterddr 



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124017823


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -48,7 +48,7 @@ public class PinotWindowExchangeNodeInsertRule extends RelOptRule {
 
   // Supported window functions
   private static final Set<SqlKind> SUPPORTED_WINDOW_FUNCTION_KIND = ImmutableSet.of(SqlKind.SUM, SqlKind.SUM0,
-      SqlKind.MIN, SqlKind.MAX, SqlKind.COUNT);
+      SqlKind.MIN, SqlKind.MAX, SqlKind.COUNT, SqlKind.OTHER_FUNCTION);

Review Comment:
   Ah good point, I think we just saw that PostgreSQL has support for all aggregation functions and thought why not add what we have (at least the basic ones). Hope it's alright to leave this here. I've updated to add a comment here.



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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1114703395


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {

Review Comment:
   nit: I think `_readyToConstruct` is not really needed since `consumeInputBlocks` indicates that no more blocks need to be consumed, and in case of error the `_upstreamErrorBlock` would be set.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1114859729


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {

Review Comment:
   Agreed that `_readyToConstruct` is not needed as such for correctness. Without using `_readyToConstruct` an extra call to `consumeInputBlocks` may be required to return the `EOS` (since today `_readyToConstruct` caches that EOS has been seen already). Not that this is a big deal, but might be okay to keep this as is for now.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124863348


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -79,7 +79,7 @@ public WindowNode(int stageId, List<Window.Group> windowGroups, List<RexLiteral>
     //       Frame literals come in the constants from the LogicalWindow and the bound.getOffset() stores the
     //       InputRef to the constants array offset by the input array length. These need to be extracted here and
     //       set to the bounds.
-    validateFrameBounds(windowGroup.lowerBound, windowGroup.upperBound, windowGroup.isRows);

Review Comment:
   never mind, updated this to add the enum directly into `WindowNode` and access it in `WindowAggregateOperator`.



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


[GitHub] [pinot] codecov-commenter commented on pull request #10286: [multistage] Initial (phase 1) Add query runtime support for empty OVER() and OVER(PARTITION BY) window functions

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10286:
URL: https://github.com/apache/pinot/pull/10286#issuecomment-1430784866

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10286](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (976df16) into [master](https://codecov.io/gh/apache/pinot/commit/addb6525fca86331dfc468209164bab1636289fb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (addb652) will **decrease** coverage by `56.68%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10286       +/-   ##
   =============================================
   - Coverage     70.39%   13.71%   -56.68%     
   + Complexity     5777      182     -5595     
   =============================================
     Files          2015     1962       -53     
     Lines        109318   107020     -2298     
     Branches      16615    16358      -257     
   =============================================
   - Hits          76954    14678    -62276     
   - Misses        26961    91186    +64225     
   + Partials       5403     1156     -4247     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `13.71% <0.00%> (-0.02%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/pinot/query/planner/StageMetadata.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcGxhbm5lci9TdGFnZU1ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-93.34%)` | :arrow_down: |
   | [...g/apache/pinot/query/planner/stage/WindowNode.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcGxhbm5lci9zdGFnZS9XaW5kb3dOb2RlLmphdmE=) | `0.00% <0.00%> (-55.11%)` | :arrow_down: |
   | [...inot/query/runtime/operator/AggregateOperator.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9BZ2dyZWdhdGVPcGVyYXRvci5qYXZh) | `0.00% <0.00%> (-92.47%)` | :arrow_down: |
   | [...uery/runtime/operator/WindowAggregateOperator.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9vcGVyYXRvci9XaW5kb3dBZ2dyZWdhdGVPcGVyYXRvci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/query/runtime/plan/PhysicalPlanVisitor.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9wbGFuL1BoeXNpY2FsUGxhblZpc2l0b3IuamF2YQ==) | `0.00% <0.00%> (-92.31%)` | :arrow_down: |
   | [...src/main/java/org/apache/pinot/sql/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvRmlsdGVyS2luZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...in/java/org/apache/pinot/spi/utils/BytesUtils.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQnl0ZXNVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/common/CustomObject.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vQ3VzdG9tT2JqZWN0LmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/core/data/table/Table.java](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1RhYmxlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1589 more](https://codecov.io/gh/apache/pinot/pull/10286?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124018814


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.

Review Comment:
   @walterddr quick question in case you have context. Why do we initialize the aggregation output without using `doubleValue()` but use `doubleValue()` in the aggregation? Just wanted to understand the context and see if I can make a fix for this in a follow up PR



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123277948


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {

Review Comment:
   yeah sure let me explore this as a future improvement



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123667643


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;
+    // Set to 'true' for ROWS type frames, otherwise set to 'false'
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+}

Review Comment:
   Got it. Makes sense



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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110298242


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   If there are no rows I think we should still return a regular `TransferableBlock` (not EOS), so that the upstream operators have at least one block that they can process, which may be needed to handle scenarios like aggregations without group-by.



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {

Review Comment:
   nit: `CollectionUtils.isEmpty`.



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {

Review Comment:
   Can you also log the exception here?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110459523


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Thanks for the suggestion @ankitsultana. Based on the plans I see that the aggregation and the aggregations with group by occur downstream (under) from the window operator. Is this still a concern in that case? Are there specific types of queries you think I should test out to check which behavior makes more sense here (`EOS` vs. empty `TransferableBlock`?
   
   I've added a few tests in my latest commit to check scenarios like empty filter or empty filter with a GROUP BY (validated that these return an empty result). Are these scenarios relevant to your suggestion here?
   
   Queries which have an aggregation and a window function but no group-by aren't allowed, such as:
   
   ```
   SELECT SUM(a.col3), SUM(a.col3) OVER() FROM a
   ```
   
   The planner throws an error:
   
   ```
   Error composing query plan for 'SELECT SUM(a.col3), SUM(a.col3) OVER() FROM a': From line 1, column 25 to line 1, column 30: Expression 'a.col3' is not being grouped'
   ```
   
   Example plan combining window function with group by aggregation (which does work):
   
   ```
           "description": "multiple OVER(PARTITION BY)s with aggregate avg and group by",
           "sql": "EXPLAIN PLAN FOR SELECT AVG(a.col3), AVG(a.col3) OVER(PARTITION BY a.col3), MAX(a.col3) OVER(PARTITION BY a.col3) FROM a GROUP BY a.col3",
           "output": [
             "Execution Plan",
             "\nLogicalProject(EXPR$0=[$1], EXPR$1=[CAST(/($2, $3)):INTEGER NOT NULL], EXPR$2=[$4])",
             "\n  LogicalWindow(window#0=[window(partition {0} aggs [SUM($0), COUNT($0), MAX($0)])])",
             "\n    LogicalExchange(distribution=[hash[0]])",
             "\n      LogicalProject(col3=[$0], EXPR$0=[CAST(/($1, $2)):INTEGER NOT NULL])",
             "\n        LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[$SUM0($2)])",
             "\n          LogicalExchange(distribution=[hash[0]])",
             "\n            LogicalAggregate(group=[{1}], agg#0=[$SUM0($1)], agg#1=[COUNT()])",
             "\n              LogicalTableScan(table=[[a]])",
             "\n"
   ```



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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117697148


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   I see. In that case either ways works for me. Can we also add this to the documentation of `MultistageOperator::nextBlock`?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110459523


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Thanks for the suggestion @ankitsultana. Based on the plans I see that the aggregation and the aggregations with group by occur downstream (under) from the window operator. Is this still a concern in that case? Are there specific types of queries you think I should test out to check which behavior makes more sense here (`EOS` vs. empty `TransferableBlock`?
   
   I've added a few tests in my latest commit to check scenarios like empty filter or empty filter with a GROUP BY (validated that these return an empty result). Are these scenarios relevant to your suggestion here?
   
   Example plan combining window function with group by aggregation:
   
   ```
           "description": "multiple OVER(PARTITION BY)s with aggregate avg and group by",
           "sql": "EXPLAIN PLAN FOR SELECT AVG(a.col3), AVG(a.col3) OVER(PARTITION BY a.col3), MAX(a.col3) OVER(PARTITION BY a.col3) FROM a GROUP BY a.col3",
           "output": [
             "Execution Plan",
             "\nLogicalProject(EXPR$0=[$1], EXPR$1=[CAST(/($2, $3)):INTEGER NOT NULL], EXPR$2=[$4])",
             "\n  LogicalWindow(window#0=[window(partition {0} aggs [SUM($0), COUNT($0), MAX($0)])])",
             "\n    LogicalExchange(distribution=[hash[0]])",
             "\n      LogicalProject(col3=[$0], EXPR$0=[CAST(/($1, $2)):INTEGER NOT NULL])",
             "\n        LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[$SUM0($2)])",
             "\n          LogicalExchange(distribution=[hash[0]])",
             "\n            LogicalAggregate(group=[{1}], agg#0=[$SUM0($1)], agg#1=[COUNT()])",
             "\n              LogicalTableScan(table=[[a]])",
             "\n"
   ```



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117708927


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/QueryRunnerTestBase.java:
##########
@@ -364,6 +371,8 @@ public static class Query {
       public List<List<Object>> _outputs = null;
       @JsonProperty("expectedException")
       public String _expectedException;
+      @JsonProperty("keepOutputRowOrder")
+      public boolean _keepOutputRowOrder;

Review Comment:
   done, ran into an issue with a few tests and left TODOs to look into them later along with comments explaining what I think is going on. As a follow up perhaps I can revisit making this 'keepOutputRowOrder' less flaky in terms of ordering (i.e. handle scenarios where the order by key is correctly ordered but there are ties)



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1119558493


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,343 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {

Review Comment:
   Yes you're right, such queries will result in more than 1 window group and is not supported as part of Phase 1. I've enhanced the planner tests to add more scenarios for multiple window groups (had just 1 earlier).



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122700244


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),

Review Comment:
   Why is 2nd check (after OR) needed ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122865754


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   Not really related to this PR but probably something we can improve generally
   
   I am thinking that if we do the processing of operators in off heap buffers, then can we do something like this to avoid memcpy ?
   
   Say we setup 3 ops in an op chain
   
   Op1 -> Op2 -> Op3 from upstream to downstream
   
   When the operators are setup, they get an input container (aka record batch with schema)
   
   Op3's input container = Op2's output container
   Op2's input container = Op1's output container
   
   When the `OpChain` is setup, we call something like setup() on operator
   
   ```
   Output setup (Input) {
      // create Output container and return it
      // the caller will use the returned Output as Input o setup the next downstream operator
   }
   ```
   
   We produce into Output but that does not require memcpy since it is direct buffers



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123269986


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.

Review Comment:
   This comment is copied over from `AggregateOperator` (since both use the `AggregationUtils`) and basically discusses that if only a single input is part of the aggregation, then the `Merger::initialize()` function today doesn't cast it as a double, so the output won't be a double. For other cases it does use the double value:
   
   ```
       default Object initialize(Object other, DataSchema.ColumnDataType dataType) {
         // TODO: Initialize as a double so that if only one row is returned it matches the type when many rows are
         //       returned
         return other == null ? dataType.getNullPlaceholder() : other;
       }
   ```
   
   Example merger function:
   
   ```
     private static Object mergeMax(Object left, Object right) {
       return Math.max(((Number) left).doubleValue(), ((Number) right).doubleValue());
     }
   ```
   
   I tried to fix this but ran into a bunch of test issues so decided to try and fix the `initialize` to also use `doubleValue()` in a separate PR if possible. Also wanted context on why the code was written in this way. Hope this clarifies the 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.

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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123332219


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY

Review Comment:
   added a TODO for multiple window groups.
   Technically the multiple window groups change should happen on the planning side where we create multiple `LogicalWindow` nodes in the tree. We can still keep a TODO here in case any operator changes are indeed needed.



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/StageMetadata.java:
##########
@@ -81,6 +82,12 @@ public void attach(StageNode stageNode) {
       _requiresSingletonInstance = _requiresSingletonInstance || (sortNode.getCollationKeys().size() > 0
           && sortNode.getOffset() != -1);
     }
+    if (stageNode instanceof WindowNode) {
+      WindowNode windowNode = (WindowNode) stageNode;
+      // Empty OVER() and OVER(ORDER BY) need to be processed on a singleton node. OVER() with PARTITION BY can be

Review Comment:
   done



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


[GitHub] [pinot] walterddr merged pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr merged PR #10286:
URL: https://github.com/apache/pinot/pull/10286


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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1119558859


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -20,6 +20,7 @@
 
 import com.clearspring.analytics.util.Preconditions;
 import java.util.ArrayList;
+import java.util.Collections;

Review Comment:
   Agreed it's not needed here. Removed it



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124017932


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -79,7 +79,7 @@ public WindowNode(int stageId, List<Window.Group> windowGroups, List<RexLiteral>
     //       Frame literals come in the constants from the LogicalWindow and the bound.getOffset() stores the
     //       InputRef to the constants array offset by the input array length. These need to be extracted here and
     //       set to the bounds.
-    validateFrameBounds(windowGroup.lowerBound, windowGroup.upperBound, windowGroup.isRows);

Review Comment:
   done



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java:
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.query.runtime.operator.utils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Utility class to perform aggregations in the intermediate stage operators such as {@code AggregateOperator} and
+ * {@code WindowAggregateOperator}
+ */
+public class AggregationUtils {

Review Comment:
   done



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123299526


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;
+    // Set to 'true' for ROWS type frames, otherwise set to 'false'
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+}

Review Comment:
   Great question. All intermediate stage operators extend from `MultiStageOperator` which has the implementation for `close()` and `cancel()`. Most intermediate stage operators don't need to override these as the same behavior is required.
   
   Code from `MultiStageOperator`:
   
   ```
     // TODO: Ideally close() call should finish within request deadline.
     // TODO: Consider passing deadline as part of the API.
     @Override
     public void close() {
       for (MultiStageOperator op : getChildOperators()) {
         try {
           op.close();
         } catch (Exception e) {
           LOGGER.error("Failed to close operator: " + op + " with exception:" + e);
           // Continue processing because even one operator failed to be close, we should still close the rest.
         }
       }
     }
   
     public void cancel(Throwable e) {
       for (MultiStageOperator op : getChildOperators()) {
         try {
           op.cancel(e);
         } catch (Exception e2) {
           LOGGER.error("Failed to cancel operator:" + op + "with error:" + e + " with exception:" + e2);
           // Continue processing because even one operator failed to be cancelled, we should still cancel the rest.
         }
       }
     }
   ```



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


[GitHub] [pinot] siddharthteotia commented on pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on PR #10286:
URL: https://github.com/apache/pinot/pull/10286#issuecomment-1449377238

   Will be reviewing this today


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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122752828


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   ^^ might just be more intuitive but nothing wrong with the current approach



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117478600


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   yup, i tried these in postgres, but since postgres does an explicit sort for even the PARTITION BY keys it did make a difference in the ordering.
   
   Since we've decided not to care about ordering on PARTITION BY key, the above 3 queries will have the same aggregation value. The NULLS LAST vs. NULLS FIRST will only modify the order of the NULL rows, but this can be handled within the operator and still treat this type of query as a PARTITION BY only query.
   
   I guess based on my own response above, adding this check probably doesn't make sense 😅 



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117709259


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {
+    Object[] keyElements = new Object[groupSet.size()];
+    for (int i = 0; i < groupSet.size(); i++) {
+      keyElements[i] = row[((RexExpression.InputRef) groupSet.get(i)).getIndex()];
+    }
+    return new Key(keyElements);
+  }
+
+  private static class OrderSetInfo {
+    final List<RexExpression> _orderSet;
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  private static class WindowFrame {
+    final int _lowerBound;
+    final int _upperBound;
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+
+  private static Object mergeSum(Object left, Object right) {
+    return ((Number) left).doubleValue() + ((Number) right).doubleValue();
+  }
+
+  private static Object mergeMin(Object left, Object right) {
+    return Math.min(((Number) left).doubleValue(), ((Number) right).doubleValue());
+  }
+
+  private static Object mergeMax(Object left, Object right) {
+    return Math.max(((Number) left).doubleValue(), ((Number) right).doubleValue());
+  }
+
+  private static class MergeCount implements WindowMerger {
+
+    @Override
+    public Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? 0 : 1d;
+    }
+
+    @Override
+    public Object merge(Object left, Object ignored) {
+      // TODO: COUNT(*) doesn't need to parse right object until we support NULL
+      return ((Number) left).doubleValue() + 1;
+    }
+  }
+
+  interface WindowMerger {
+    /**
+     * Initializes the merger based on the first input
+     */
+    default Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? ((Number) dataType.getNullPlaceholder()).doubleValue() : ((Number) other).doubleValue();
+    }
+
+    /**
+     * Merges the existing aggregate (the result of {@link #initialize(Object, DataSchema.ColumnDataType)}) with
+     * the new value coming in (which may be an aggregate in and of itself).
+     */
+    Object merge(Object agg, Object value);
+  }
+
+  private static class WindowAccumulator {
+    private static final Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> WINDOW_MERGERS =
+        ImmutableMap.<String, Function<DataSchema.ColumnDataType, WindowMerger>>builder()
+            .put("SUM", cdt -> WindowAggregateOperator::mergeSum)
+            .put("$SUM", cdt -> WindowAggregateOperator::mergeSum)
+            .put("$SUM0", cdt -> WindowAggregateOperator::mergeSum)
+            .put("MIN", cdt -> WindowAggregateOperator::mergeMin)
+            .put("$MIN", cdt -> WindowAggregateOperator::mergeMin)
+            .put("$MIN0", cdt -> WindowAggregateOperator::mergeMin)
+            .put("MAX", cdt -> WindowAggregateOperator::mergeMax)
+            .put("$MAX", cdt -> WindowAggregateOperator::mergeMax)
+            .put("$MAX0", cdt -> WindowAggregateOperator::mergeMax)
+            .put("COUNT", cdt -> new MergeCount())
+            .build();

Review Comment:
   done. should have done this right from the get go :) 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {
+    Object[] keyElements = new Object[groupSet.size()];
+    for (int i = 0; i < groupSet.size(); i++) {
+      keyElements[i] = row[((RexExpression.InputRef) groupSet.get(i)).getIndex()];
+    }
+    return new Key(keyElements);
+  }
+
+  private static class OrderSetInfo {
+    final List<RexExpression> _orderSet;
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  private static class WindowFrame {

Review Comment:
   done. yes your understanding of isRows is indeed ROWS vs. RANGE



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117707764


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java:
##########
@@ -308,6 +303,20 @@ public Object merge(Object agg, Object value) {
     }
   }
 
+  private static class MergeCounts implements Merger {
+
+    @Override
+    public Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? 0 : 1;
+    }
+
+    @Override
+    public Object merge(Object left, Object ignored) {
+      // TODO: COUNT(*) doesn't need to parse right object until we support NULL
+      return ((Number) left).doubleValue() + 1;

Review Comment:
   done



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1107404202


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   I'm not entirely sure if this is actually required, but thought I'd look into the correct semantics once I start the work on `ORDER BY` within `OVER()`. This check will prevent marking queries like below as partition by only queries:
   
   - `OVER(PARTITION BY key1 ORDER BY key1 DESC)`
   - `OVER(PARTITION BY key1 ORDER BY key1 NULLS FIRST)`
   
   The code will however allow queries of the type `OVER(PARTITION BY key1 ORDER BY key1)` to be marked as partition by only (since all aggregated values will be the same across the whole partition just like `OVER(PARTITION BY)`).
   
   Let me know if the preference is to just remove this extra check since logically the actual aggregation values will still be the same for the full partition irrespective of the direction and null direction.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123311802


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {

Review Comment:
   Keeping the name the same to be consistent with `AggregateOperator` and `SortOperator`



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123330400


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);

Review Comment:
   We also support BOOL_OR and BOOL_AND
   Updated the error message



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {

Review Comment:
   done



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123330837


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)

Review Comment:
   done



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123666979


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   Sounds good. FYI @walterddr 



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124018333


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;

Review Comment:
   Agree! we'll need to test this out for performance for sure



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


[GitHub] [pinot] somandal commented on pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on PR #10286:
URL: https://github.com/apache/pinot/pull/10286#issuecomment-1452962243

   > lgtm with some more comments. thank you @somandal for the contribution. this is a huge effort and we definitely appreciate the patience and consideration during the review/comment process.
   
   Thanks @walterddr! Appreciate the thorough review and the guidance on this from all involved.
   I've addressed all comments. PTAL when you get a chance


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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1113843243


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   thanks for clarifying, done!



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117678564


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   I dug into this a bit more. @ankitsultana please confirm if the scenario I'm going to discuss here matches the scenario you mentioned or not.
   
   So from what I understood, @ankitsultana wanted an empty block to be returned for queries with sub queries with the window function inside the sub query such as:
   
   ```
         {
           "sql": "SELECT SUM(count) FROM (SELECT string_col, COUNT(bool_col) OVER() as count FROM {tbl} WHERE string_col = 'a' AND bool_col = false AND int_col > 200)",
           "outputs": [
             [0]
           ]
         },
   ```
   
   In scenarios where the window inside the sub-query returns 0 rows, the outer query's aggregation should still work and return a result of '0' instead of no rows.
   
   Now I actually tried the above (and will add tests with my latest commit) and found that it doesn't matter if I return an empty row or EOS here. In both scenarios the outer query aggregation returns the correct result.
   
   Based on this @ankitsultana and @walterddr can you confirm if I can change this back to return EOS when the number of rows is 0?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122746502


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {

Review Comment:
   (nit) suggest renaming this to `produceOutputBlock`



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122751054


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   The state that whether or not we can produce output block should be maintained and dictated by the piece of code that consumed the input block - `consumeInputBlocks`
   
   Generally speaking, whenever an operator consumes an input block, before exiting that function it should be able to determine whether or not it is ready to produce output and accordingly change the local state which can then be leveraged in `getNextBlock`.
   
   



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123288336


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),

Review Comment:
   This is a temporary check because today we don't support window functions with `ORDER BY` where the `ORDER BY` key is different from the `PARTITION BY` key. We do support queries where the keys are the same, e.g. `OVER(PARTITION BY key1 ORDER BY key1)`.
   
   For queries such as `OVER(PARTITION BY key1 ORDER BY key1)`, the default frame lands up getting set to `UNBOUNDED PRECEDING` to `CURRENT ROW`. This check here ensures that if the `upperBound` is indeed `CURRENT ROW` we should check that it's  a query where the `PARTITION BY` and `ORDER BY` key is the same.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123669142


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());

Review Comment:
   Great. Thanks



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122705783


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;

Review Comment:
   (nit) multi stage engine's query planning is likely to be heap heavy so we may want to double check if `int` is really needed or can we do away with `short`. 
   
   not a big deal in this PR but something to do in follow-up if folks agree



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123997504


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -48,7 +48,7 @@ public class PinotWindowExchangeNodeInsertRule extends RelOptRule {
 
   // Supported window functions
   private static final Set<SqlKind> SUPPORTED_WINDOW_FUNCTION_KIND = ImmutableSet.of(SqlKind.SUM, SqlKind.SUM0,
-      SqlKind.MIN, SqlKind.MAX, SqlKind.COUNT);
+      SqlKind.MIN, SqlKind.MAX, SqlKind.COUNT, SqlKind.OTHER_FUNCTION);

Review Comment:
   what is the other function we've supported? can we add a comment for this? (i think it is BOOL_AND and BOOL_OR)
   
   but IMO, BOOL_AND and BOOL_OR are just MIN(booleanCol) and MAX(booleanCol) :-P i didn't understand why we needed that in the first place lol



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -79,7 +79,7 @@ public WindowNode(int stageId, List<Window.Group> windowGroups, List<RexLiteral>
     //       Frame literals come in the constants from the LogicalWindow and the bound.getOffset() stores the
     //       InputRef to the constants array offset by the input array length. These need to be extracted here and
     //       set to the bounds.
-    validateFrameBounds(windowGroup.lowerBound, windowGroup.upperBound, windowGroup.isRows);

Review Comment:
   commented previously not sure isRows is a clear enough flag. let's name it `isRowBased` or use a enum 
   ```
   enum WindowBase {
     ROW, RANGE
   }
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java:
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.query.runtime.operator.utils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Utility class to perform aggregations in the intermediate stage operators such as {@code AggregateOperator} and
+ * {@code WindowAggregateOperator}
+ */
+public class AggregationUtils {

Review Comment:
   SWEET refactoring!!! Thank you!!!
   
   suggest javadoc
   
   ```suggestion
   /**
    * Utility class to perform accumulation over a collection of rows. It provides util to 
    * (1) method to deal with aggregation key and (2) method to merge a row into an existing accumulator
    *
    * <p>Accumulation is used by {@code WindowAggregateOperator} and {@code AggregateOperator}.
    */
   public class AggregationUtils {
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   i dont understand why we need to copy in the first place. window over is 1-in-1-out in the row perspective. it only caches the row it gets, accumulate the results for the additional columns, and attach the result back to the row when it finishes. 
   
   i agree we will have overhead for heap memory b/c we need to buffer everything. but not sure why we need a mem copy, it felt like if we are using `List<Object>` instead of `Object[]` we can completely avoid this copy yes?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;

Review Comment:
   let's follow up later but IMO the most heavy part is not the memory size but the latency overhead :-P



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   for now the logic is kind of there to guard the EOS as @somandal mentioned. we can revisit in general later. please kindly file an issue. (I am yet to understand what's the confusion and how can we improve it, but will look into it more generally once we file the issue with more context)



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;
+    // Set to 'true' for ROWS type frames, otherwise set to 'false'
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+}

Review Comment:
   correct. no need to do explicit close unless these are non-automatic GC clean up such as GRPC mailbox operator which will have dangling resources.



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117953759


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   correct. the tradition of Postgres (as far as I know), is that all aggregate without group should return 1 row. for example
   ```
   SELECT * FROM tbl WHERE val > (SELECT MAX(val) FROM tbl2)
   SELECT * FROM tbl WHERE (SELECT count(val) FROM tbl2) > 0
   ```
   both example the SQL relational planner interprets the inner query result as a single value. this can only happen if the agg without group by operators as I mentioned above.
   
   So i think this rule doesn't apply to window, please share more if the analysis above can be in any way improved



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1119558630


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,343 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include

Review Comment:
   Yes correct, added AVG to the list



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117700153


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   I'm not sure if adding documentation on this is really necessary. The above works because the outer query uses an intermediate AggregateOperator which does the right thing (i.e. @walterddr 's fix that we discussed 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.

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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110459523


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Thanks for the suggestion @ankitsultana. Based on the plans I see that the aggregation and the aggregations with group by occur downstream (under) from the window operator. Is this still a concern in that case? Are there specific types of queries you think I should test out to check which behavior makes more sense here (`EOS` vs. empty `TransferableBlock`?
   
   I've added a few tests in my latest commit to check scenarios like empty filter or empty filter with a GROUP BY (validated that these return an empty result). Are these scenarios relevant to your suggestion here?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1119559292


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   done, removed this check. I'll add special handling for null direction later on (left a TODO). This will only affect the order in which the columns with NULL values are output in the final result.



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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1113808120


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   I think you can pass in an empty container. `TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW)`.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117709600


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {

Review Comment:
   done



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122648627


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/StageMetadata.java:
##########
@@ -81,6 +82,12 @@ public void attach(StageNode stageNode) {
       _requiresSingletonInstance = _requiresSingletonInstance || (sortNode.getCollationKeys().size() > 0
           && sortNode.getOffset() != -1);
     }
+    if (stageNode instanceof WindowNode) {
+      WindowNode windowNode = (WindowNode) stageNode;
+      // Empty OVER() and OVER(ORDER BY) need to be processed on a singleton node. OVER() with PARTITION BY can be

Review Comment:
   > Empty OVER() and OVER(ORDER BY) need to be processed on a singleton node.
   
   Add a TODO to improve this in future per original discussion ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122687966


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");

Review Comment:
   Hmm not sure I understand this. I thought FRAME is not supported in the initial implementation based on the phases laid out in design doc. 



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122702717


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);

Review Comment:
   So this is when anything other than SUM / MIN / MAX / COUNT / AVG is used ?
   
   Might want to use a slightly more self understandable error message
   
   



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122702262


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());

Review Comment:
   Also such error can be thrown sooner than operator creation point imo ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122663629


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY

Review Comment:
   Do we need to add a TODO bullet point for multiple `OVER()` with different `PARTITION BY `and `ORDER BY` ?
   
   On a related note, you may want to update the PR description that changes here also support multiple `OVER()` with `PARTITION BY` on the same column ?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123329582


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {

Review Comment:
   done



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);

Review Comment:
   done



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1107404202


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   I'm not entirely sure if this is actually required, but thought I'd look into the correct semantics once I start the work on `ORDER BY` within `OVER()`. This check will prevent marking queries like below as partition by only queries:
   
   - OVER(PARTITION BY key1 ORDER BY key1 DESC)
   - OVER(PARTITION BY key1 ORDER BY key1 NULLS FIRST)
   
   The code will however allow queries of the type `OVER(PARTITION BY key1 ORDER BY key1)` to be marked as partition by only (since all aggregated values will be the same across the whole partition just like OVER(PARTITION BY)).
   
   Let me know if the preference is to just remove this extra check since logically the actual aggregation values will still be the same for the full partition irrespective of the direction and null direction.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122700752


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {

Review Comment:
   (nit) may be do `aggCalls.size()` once before the loop ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122865754


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   Not really related to this PR but probably something we can improve generally
   
   I am thinking that if we do the processing of operators in off heap buffers, then can we do something like this to avoid memcpy ?
   
   Say we setup 3 ops in an op chain
   
   Op1 -> Op2 -> Op3 from upstream to downstream
   
   When the operators are setup, they get an input container (aka record batch with schema)
   
   Op3's input container = Op2's output container
   Op2's input container = Op1's output container
   
   When the `OpChain` is setup, we call something like setup() on operator
   
   Output setup (Input) {
      // create Output container and return it
      // the caller will use the returned Output as Input o setup the next downstream operator
   }
   
   We produce into Output but that does not require memcpy since it is direct buffers



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122665421


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)

Review Comment:
   (nit) might want to capture TODOs in one place.. seem to be repeated here as everything is already called out in the constructor JavaDoc ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1122704055


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added, also revisit null direction
+        //       handling for all query types
+        Key key = AggregationUtils.extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  /**
+   * Contains all the ORDER BY key related information such as the keys, direction, and null direction
+   */
+  private static class OrderSetInfo {
+    // List of order keys
+    final List<RexExpression> _orderSet;
+    // List of order direction for each key
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    // List of null direction for each key
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  /**
+   * Defines the Frame to be used for the window query. The 'lowerBound' and 'upperBound' indicate the frame
+   * boundaries to be used. Whereas, 'isRows' is used to differentiate between RANGE and ROWS type frames.
+   */
+  private static class WindowFrame {
+    // The lower bound of the frame. Set to Integer.MIN_VALUE if UNBOUNDED PRECEDING
+    final int _lowerBound;
+    // The lower bound of the frame. Set to Integer.MAX_VALUE if UNBOUNDED FOLLOWING. Set to 0 if CURRENT ROW
+    final int _upperBound;
+    // Set to 'true' for ROWS type frames, otherwise set to 'false'
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+}

Review Comment:
   Don't we also need to implement close() to cleanup any resources ? IIRC, recently all operators were changed to extend from `AutoCloseable` and therefore should try to override close() if needed



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124017932


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/planner/stage/WindowNode.java:
##########
@@ -79,7 +79,7 @@ public WindowNode(int stageId, List<Window.Group> windowGroups, List<RexLiteral>
     //       Frame literals come in the constants from the LogicalWindow and the bound.getOffset() stores the
     //       InputRef to the constants array offset by the input array length. These need to be extracted here and
     //       set to the bounds.
-    validateFrameBounds(windowGroup.lowerBound, windowGroup.upperBound, windowGroup.isRows);

Review Comment:
   done - I renamed it here, and store it as an enum in `WindowAggregateOperator`. If it's okay I'll create a util to store the enum in a common place for use in both places later as a followup?



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123308485


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i].getResults().get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;

Review Comment:
   I've tried to keep the operator flow similar to `AggregateOperator` for now. This flag `_hasReturnedWindowAggregateBlock` is meant to differentiate the scenario where we finished consumption and haven't yet sent the output block, vs. the scenario where we finished consumption and sent the output block already but now just need to send a final `EOS`. In both cases the `consumeInputBlocks` will indicate there is nothing more to send.
   
   We can revisit this logic for all operators at once if needed.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123311117


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   Great suggestion. Let's look into this as a separate change.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123290056


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());

Review Comment:
   These checks are done in the planner while building the calcite plan. I have a planner test too to ensure that `ROW_NUMBER` etc throw.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1123282327


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");

Review Comment:
   These checks are for default frames (just to ensure we aren't getting inputs where the frames are overridden)
   
   - Without `ORDER BY`: default frame: `UNBOUNDED PRECEDING` to `UNBOUNDED FOLLOWING`
   - With `ORDER BY`: default frame: `UNBOUNDED PRECEDING` to `CURRENT ROW`
   
   Also whether `ROWS` or `RANGE` is the default depends on the default frame so that's why I check for that too.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1124009602


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,332 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.utils.AggregationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX/AVG/BOOL_OR/BOOL_AND aggregations. Window functions also
+ * include other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * Note: This class performs aggregation over the double value of input.
+ * If the input is single value, the output type will be input type. Otherwise, the output type will be double.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ *     5. Add support for null direction handling (even for PARTITION BY only queries with custom null direction)
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final AggregationUtils.Accumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, AggregationUtils.Accumulator.MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, AggregationUtils.Merger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new AggregationUtils.Accumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new AggregationUtils.Accumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    return partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);

Review Comment:
   We need to create an output in the format of `List<Object[]>`, where each `Object[]` contains the contents of a single row. The output `Object[]` row size is larger than the input row's size since we need to store the aggregated columns. Using `List<Object>` instead of `Object[]` can avoid the copy as it can be appended to and added to the output result.



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1110459523


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   thanks for the suggestion @ankitsultana. Based on the plans I see that the aggregation and the aggregations with group by occur downstream (under) from the window operator. Is this still a concern in that case? Are there specific types of queries you think I should test out to check which behavior makes more sense here (`EOS` vs. empty `TransferableBlock`?
   
   I've added a few tests in my latest commit to check scenarios like empty filter or empty filter with a GROUP BY (validated that these return an empty result). Are these scenarios relevant to your suggestion here?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {

Review Comment:
   done



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1113346747


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   Got it, thanks for explaining the context here. I did look at @walterddr's change to make agg operators return a non-EOS response when rows is empty. This is only done when the group by set is empty (i.e. aggregation without group by query).
   
   ```
       if (rows.size() == 0) {
         if (_groupSet.size() == 0) {
           return constructEmptyAggResultBlock();
         } else {
           return TransferableBlockUtils.getEndOfStreamTransferableBlock();
         }
       } else {
         return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
       }
   ```
   
   My worry here when it come to returning an empty block is what should I fill in for the values of other columns (e.g. the select columns) for this scenario? Or do you think perhaps we should only return an empty block if there are no other input columns? cc @walterddr for any thoughts on this as well.
   
   e.g. query where I need to worry about what to fill in for `a.col1`:
   
   `SELECT a.col1, SUM(a.col3) OVER(PARTITION BY a.col1) FROM table a;`



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117953759


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   correct. the tradition of Postgres (as far as I know), is that all aggregate without group should return 1 row. for example
   ```
   SELECT * FROM tbl WHERE val > (SELECT MAX(val) FROM tbl2)
   SELECT * FROM tbl WHERE (SELECT count(val) FROM tbl2) > 0
   ```
   both example the SQL relational planner interprets the inner query result as a single value. this can only happen if the agg without group by operators as I mentioned above (always return 1 row, and both inner query has 1 column)
   
   So i think this rule doesn't apply to window, please share more if the analysis above can be in any way improved



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117478600


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   yup, i tried these in postgres, but since postgres does an explicit sort for even the PARTITION BY keys it did make a difference in the ordering. The aggregation value for the full partition remained the same though
   
   Since we've decided not to care about ordering on PARTITION BY key, the above 3 queries will have the same aggregation value. The NULLS LAST vs. NULLS FIRST will only modify the order of the NULL rows, but this can be handled within the operator and still treat this type of query as a PARTITION BY only query.
   
   I guess based on my own response above, adding this check probably doesn't make sense 😅 



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


[GitHub] [pinot] somandal commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "somandal (via GitHub)" <gi...@apache.org>.
somandal commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117685201


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   @walterddr should I go ahead and remove this check?



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10286: [multistage] Initial (phase 1) Query runtime for window functions - empty OVER() and OVER(PARTITION BY)

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10286:
URL: https://github.com/apache/pinot/pull/10286#discussion_r1117427355


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java:
##########
@@ -308,6 +303,20 @@ public Object merge(Object agg, Object value) {
     }
   }
 
+  private static class MergeCounts implements Merger {
+
+    @Override
+    public Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? 0 : 1;
+    }
+
+    @Override
+    public Object merge(Object left, Object ignored) {
+      // TODO: COUNT(*) doesn't need to parse right object until we support NULL
+      return ((Number) left).doubleValue() + 1;

Review Comment:
   this should be supported right?
   ```suggestion
         return ((Number) left).doubleValue() + (ignored == null ? 0 : 1);
   ```
   and change `ignored` to `right`



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {
+    Object[] keyElements = new Object[groupSet.size()];
+    for (int i = 0; i < groupSet.size(); i++) {
+      keyElements[i] = row[((RexExpression.InputRef) groupSet.get(i)).getIndex()];
+    }
+    return new Key(keyElements);
+  }
+
+  private static class OrderSetInfo {
+    final List<RexExpression> _orderSet;
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  private static class WindowFrame {
+    final int _lowerBound;
+    final int _upperBound;
+    final boolean _isRows;
+
+    WindowFrame(int lowerBound, int upperBound, boolean isRows) {
+      _lowerBound = lowerBound;
+      _upperBound = upperBound;
+      _isRows = isRows;
+    }
+
+    boolean isUnboundedPreceding() {
+      return _lowerBound == Integer.MIN_VALUE;
+    }
+
+    boolean isUnboundedFollowing() {
+      return _upperBound == Integer.MAX_VALUE;
+    }
+
+    boolean isUpperBoundCurrentRow() {
+      return _upperBound == 0;
+    }
+
+    boolean isRows() {
+      return _isRows;
+    }
+
+    int getLowerBound() {
+      return _lowerBound;
+    }
+
+    int getUpperBound() {
+      return _upperBound;
+    }
+  }
+
+  private static Object mergeSum(Object left, Object right) {
+    return ((Number) left).doubleValue() + ((Number) right).doubleValue();
+  }
+
+  private static Object mergeMin(Object left, Object right) {
+    return Math.min(((Number) left).doubleValue(), ((Number) right).doubleValue());
+  }
+
+  private static Object mergeMax(Object left, Object right) {
+    return Math.max(((Number) left).doubleValue(), ((Number) right).doubleValue());
+  }
+
+  private static class MergeCount implements WindowMerger {
+
+    @Override
+    public Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? 0 : 1d;
+    }
+
+    @Override
+    public Object merge(Object left, Object ignored) {
+      // TODO: COUNT(*) doesn't need to parse right object until we support NULL
+      return ((Number) left).doubleValue() + 1;
+    }
+  }
+
+  interface WindowMerger {
+    /**
+     * Initializes the merger based on the first input
+     */
+    default Object initialize(Object other, DataSchema.ColumnDataType dataType) {
+      return other == null ? ((Number) dataType.getNullPlaceholder()).doubleValue() : ((Number) other).doubleValue();
+    }
+
+    /**
+     * Merges the existing aggregate (the result of {@link #initialize(Object, DataSchema.ColumnDataType)}) with
+     * the new value coming in (which may be an aggregate in and of itself).
+     */
+    Object merge(Object agg, Object value);
+  }
+
+  private static class WindowAccumulator {
+    private static final Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> WINDOW_MERGERS =
+        ImmutableMap.<String, Function<DataSchema.ColumnDataType, WindowMerger>>builder()
+            .put("SUM", cdt -> WindowAggregateOperator::mergeSum)
+            .put("$SUM", cdt -> WindowAggregateOperator::mergeSum)
+            .put("$SUM0", cdt -> WindowAggregateOperator::mergeSum)
+            .put("MIN", cdt -> WindowAggregateOperator::mergeMin)
+            .put("$MIN", cdt -> WindowAggregateOperator::mergeMin)
+            .put("$MIN0", cdt -> WindowAggregateOperator::mergeMin)
+            .put("MAX", cdt -> WindowAggregateOperator::mergeMax)
+            .put("$MAX", cdt -> WindowAggregateOperator::mergeMax)
+            .put("$MAX0", cdt -> WindowAggregateOperator::mergeMax)
+            .put("COUNT", cdt -> new MergeCount())
+            .build();

Review Comment:
   let's extract out these to a common `Accumulator` interface and reused in both agg operator and window operator



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {

Review Comment:
   same here please make a comment Agg util for keeping these utils 
   we can even have a separate PR to just refactor these out



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {

Review Comment:
   let's keep it this way. 
   the `readyToConstruct` flag is only here to ensure correctness. for example if the data is already sorted and we are doing partition by order by, then we can actually produce rows as we go. 
   
   let's revisit this later



##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/QueryRunnerTestBase.java:
##########
@@ -364,6 +371,8 @@ public static class Query {
       public List<List<Object>> _outputs = null;
       @JsonProperty("expectedException")
       public String _expectedException;
+      @JsonProperty("keepOutputRowOrder")
+      public boolean _keepOutputRowOrder;

Review Comment:
   good catch. would you be able to also change the OrderBy.json and add the keepOutputRowOrder flag?



##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java:
##########
@@ -160,6 +162,19 @@ private boolean isPartitionByOnlyQuery(Window.Group windowGroup) {
       Set<Integer> partitionByKeyList = new HashSet<>(windowGroup.keys.toList());
       Set<Integer> orderByKeyList = new HashSet<>(windowGroup.orderKeys.getKeys());
       isPartitionByOnly = partitionByKeyList.equals(orderByKeyList);
+      if (isPartitionByOnly) {

Review Comment:
   all 3 of the example above produces the exact same query results right? 
   



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,431 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId) {
+    super(requestId, stageId);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (orderSet == null || orderSet.isEmpty()) {
+      return true;
+    }
+
+    if (groupSet == null || groupSet.isEmpty() || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {

Review Comment:
   i am not sure i follow the discussion here. what exact query are we trying to mitigate?
   using postgres, none of these query returns a single row
   ```
   SELECT id, SUM(val) OVER() FROM test where 0 = 1
   SELECT id, SUM(val) OVER(PARTITION BY id) FROM test where 0 = 1
   SELECT id, SUM(val) OVER(PARTITION BY id ORDER BY val) FROM test where 0 =1
   SELECT id, SUM(val) OVER(PARTITION BY id ORDER BY id) FROM test where 0 = 1
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java:
##########
@@ -0,0 +1,436 @@
+/**
+ * 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.query.runtime.operator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.pinot.common.datablock.DataBlock;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.query.planner.logical.RexExpression;
+import org.apache.pinot.query.routing.VirtualServerAddress;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The WindowAggregateOperator is used to compute window function aggregations over a set of optional
+ * PARTITION BY keys, ORDER BY keys and a FRAME clause. The output data will include the projected
+ * columns and in addition will add the aggregation columns to the output data.
+ * [input columns, aggregate result1, ... aggregate resultN]
+ *
+ * The window functions supported today are SUM/COUNT/MIN/MAX aggregations. Window functions also include
+ * other types of functions such as rank and value functions.
+ *
+ * Unlike the AggregateOperator which will output one row per group, the WindowAggregateOperator
+ * will output as many rows as input rows.
+ *
+ * TODO:
+ *     1. Add support for OVER() clause with ORDER BY only or PARTITION BY ORDER BY
+ *     2. Add support for rank window functions
+ *     3. Add support for value window functions
+ *     4. Add support for custom frames
+ */
+public class WindowAggregateOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "WINDOW";
+  private static final Logger LOGGER = LoggerFactory.getLogger(WindowAggregateOperator.class);
+
+  private final MultiStageOperator _inputOperator;
+  private final List<RexExpression> _groupSet;
+  private final OrderSetInfo _orderSetInfo;
+  private final WindowFrame _windowFrame;
+  private final List<RexExpression.FunctionCall> _aggCalls;
+  private final List<RexExpression> _constants;
+  private final DataSchema _resultSchema;
+  private final WindowAccumulator[] _windowAccumulators;
+  private final Map<Key, List<Object[]>> _partitionRows;
+
+  private TransferableBlock _upstreamErrorBlock;
+
+  private int _numRows;
+  private boolean _readyToConstruct;
+  private boolean _hasReturnedWindowAggregateBlock;
+
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      long requestId, int stageId, VirtualServerAddress virtualServerAddress) {
+    this(inputOperator, groupSet, orderSet, orderSetDirection, orderSetNullDirection, aggCalls, lowerBound,
+        upperBound, isRows, constants, resultSchema, inputSchema, WindowAccumulator.WINDOW_MERGERS,
+        requestId, stageId, virtualServerAddress);
+  }
+
+  @VisibleForTesting
+  public WindowAggregateOperator(MultiStageOperator inputOperator, List<RexExpression> groupSet,
+      List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection, List<RexExpression> aggCalls, int lowerBound,
+      int upperBound, boolean isRows, List<RexExpression> constants, DataSchema resultSchema, DataSchema inputSchema,
+      Map<String, Function<DataSchema.ColumnDataType, WindowMerger>> mergers, long requestId, int stageId,
+      VirtualServerAddress virtualServerAddress) {
+    super(requestId, stageId, virtualServerAddress);
+
+    boolean isPartitionByOnly = isPartitionByOnlyQuery(groupSet, orderSet, orderSetDirection, orderSetNullDirection);
+    // TODO: add support for ORDER BY in the OVER() clause
+    Preconditions.checkState(orderSet == null || orderSet.isEmpty() || isPartitionByOnly,
+        "Order by is not yet supported in window functions");
+
+    _inputOperator = inputOperator;
+    _groupSet = groupSet;
+    _orderSetInfo = new OrderSetInfo(orderSet, orderSetDirection, orderSetNullDirection);
+    _windowFrame = new WindowFrame(lowerBound, upperBound, isRows);
+
+    // TODO: add support for custom frames, and for ORDER BY default frame (upperBound => currentRow)
+    Preconditions.checkState(!_windowFrame.isRows(), "Only RANGE type frames are supported at present");
+    Preconditions.checkState(_windowFrame.isUnboundedPreceding(),
+        "Only default frame is supported, lowerBound must be UNBOUNDED PRECEDING");
+    Preconditions.checkState(_windowFrame.isUnboundedFollowing()
+            || (_windowFrame.isUpperBoundCurrentRow() && isPartitionByOnly),
+        "Only default frame is supported, upperBound must be UNBOUNDED FOLLOWING or CURRENT ROW");
+
+    // we expect all agg calls to be aggregate function calls
+    _aggCalls = aggCalls.stream().map(RexExpression.FunctionCall.class::cast).collect(Collectors.toList());
+    _constants = constants;
+    _resultSchema = resultSchema;
+
+    // TODO: Not all window functions (e.g. ROW_NUMBER, LAG, etc) need aggregations. Such functions should be handled
+    //       differently.
+    _windowAccumulators = new WindowAccumulator[_aggCalls.size()];
+    for (int i = 0; i < _aggCalls.size(); i++) {
+      RexExpression.FunctionCall agg = _aggCalls.get(i);
+      String functionName = agg.getFunctionName();
+      if (!mergers.containsKey(functionName)) {
+        throw new IllegalStateException("Unexpected value: " + functionName);
+      }
+      _windowAccumulators[i] = new WindowAccumulator(agg, mergers, functionName, inputSchema);
+    }
+
+    _partitionRows = new HashMap<>();
+
+    _numRows = 0;
+    _readyToConstruct = false;
+    _hasReturnedWindowAggregateBlock = false;
+  }
+
+  @Override
+  public List<MultiStageOperator> getChildOperators() {
+    return ImmutableList.of(_inputOperator);
+  }
+
+  @Nullable
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME;
+  }
+
+  @Override
+  protected TransferableBlock getNextBlock() {
+    try {
+      if (!_readyToConstruct && !consumeInputBlocks()) {
+        return TransferableBlockUtils.getNoOpTransferableBlock();
+      }
+
+      if (_upstreamErrorBlock != null) {
+        return _upstreamErrorBlock;
+      }
+
+      if (!_hasReturnedWindowAggregateBlock) {
+        return produceWindowAggregateBlock();
+      } else {
+        // TODO: Move to close call.
+        return TransferableBlockUtils.getEndOfStreamTransferableBlock();
+      }
+    } catch (Exception e) {
+      LOGGER.error("Caught exception while executing WindowAggregationOperator, returning an error block", e);
+      return TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+  }
+
+  private boolean isPartitionByOnlyQuery(List<RexExpression> groupSet, List<RexExpression> orderSet,
+      List<RelFieldCollation.Direction> orderSetDirection,
+      List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+    if (CollectionUtils.isEmpty(orderSet)) {
+      return true;
+    }
+
+    if (CollectionUtils.isEmpty(groupSet) || (groupSet.size() != orderSet.size())) {
+      return false;
+    }
+
+    Set<Integer> partitionByInputRefIndexes = new HashSet<>();
+    Set<Integer> orderByInputRefIndexes = new HashSet<>();
+    for (int i = 0; i < groupSet.size(); i++) {
+      partitionByInputRefIndexes.add(((RexExpression.InputRef) groupSet.get(i)).getIndex());
+      orderByInputRefIndexes.add(((RexExpression.InputRef) orderSet.get(i)).getIndex());
+    }
+
+    boolean isPartitionByOnly = partitionByInputRefIndexes.equals(orderByInputRefIndexes);
+    if (isPartitionByOnly) {
+      // Check the direction and null direction to ensure default ordering on the order by keys, which are:
+      // Direction: ASC
+      // Null Direction: LAST
+      for (int i = 0; i < orderSet.size(); i++) {
+        if (orderSetDirection.get(i) == RelFieldCollation.Direction.DESCENDING
+            || orderSetNullDirection.get(i) == RelFieldCollation.NullDirection.FIRST) {
+          isPartitionByOnly = false;
+          break;
+        }
+      }
+    }
+    return isPartitionByOnly;
+  }
+
+  private TransferableBlock produceWindowAggregateBlock() {
+    List<Object[]> rows = new ArrayList<>(_numRows);
+    for (Map.Entry<Key, List<Object[]>> e : _partitionRows.entrySet()) {
+      Key partitionKey = e.getKey();
+      List<Object[]> rowList = e.getValue();
+      for (Object[] existingRow : rowList) {
+        Object[] row = new Object[existingRow.length + _aggCalls.size()];
+        System.arraycopy(existingRow, 0, row, 0, existingRow.length);
+        for (int i = 0; i < _windowAccumulators.length; i++) {
+          row[i + existingRow.length] = _windowAccumulators[i]._results.get(partitionKey);
+        }
+        rows.add(row);
+      }
+    }
+    _hasReturnedWindowAggregateBlock = true;
+    if (rows.size() == 0) {
+      return new TransferableBlock(Collections.emptyList(), _resultSchema, DataBlock.Type.ROW);
+    } else {
+      return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
+    }
+  }
+
+  /**
+   * @return whether or not the operator is ready to move on (EOS or ERROR)
+   */
+  private boolean consumeInputBlocks() {
+    TransferableBlock block = _inputOperator.nextBlock();
+    while (!block.isNoOpBlock()) {
+      // setting upstream error block
+      if (block.isErrorBlock()) {
+        _upstreamErrorBlock = block;
+        return true;
+      } else if (block.isEndOfStreamBlock()) {
+        _readyToConstruct = true;
+        return true;
+      }
+
+      List<Object[]> container = block.getContainer();
+      for (Object[] row : container) {
+        _numRows++;
+        // TODO: Revisit the aggregation logic once ORDER BY inside OVER() support is added
+        Key key = extractRowKey(row, _groupSet);
+        _partitionRows.putIfAbsent(key, new ArrayList<>());
+        _partitionRows.get(key).add(row);
+        for (int i = 0; i < _aggCalls.size(); i++) {
+          _windowAccumulators[i].accumulate(key, row);
+        }
+      }
+      block = _inputOperator.nextBlock();
+    }
+    return false;
+  }
+
+  private static Key extractRowKey(Object[] row, List<RexExpression> groupSet) {
+    Object[] keyElements = new Object[groupSet.size()];
+    for (int i = 0; i < groupSet.size(); i++) {
+      keyElements[i] = row[((RexExpression.InputRef) groupSet.get(i)).getIndex()];
+    }
+    return new Key(keyElements);
+  }
+
+  private static class OrderSetInfo {
+    final List<RexExpression> _orderSet;
+    final List<RelFieldCollation.Direction> _orderSetDirection;
+    final List<RelFieldCollation.NullDirection> _orderSetNullDirection;
+
+    OrderSetInfo(List<RexExpression> orderSet, List<RelFieldCollation.Direction> orderSetDirection,
+        List<RelFieldCollation.NullDirection> orderSetNullDirection) {
+      _orderSet = orderSet;
+      _orderSetDirection = orderSetDirection;
+      _orderSetNullDirection = orderSetNullDirection;
+    }
+
+    List<RexExpression> getOrderSet() {
+      return _orderSet;
+    }
+
+    List<RelFieldCollation.Direction> getOrderSetDirection() {
+      return _orderSetDirection;
+    }
+
+    List<RelFieldCollation.NullDirection> getOrderSetNullDirection() {
+      return _orderSetNullDirection;
+    }
+  }
+
+  private static class WindowFrame {

Review Comment:
   javadoc please, i am not sure what isRows mean --> are we using this to determine this RANGE-based or ROW-based?



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