You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/03/12 18:53:37 UTC

[GitHub] [incubator-pinot] mqliang opened a new pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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


   This is exactly the same as: https://github.com/apache/incubator-pinot/pull/6670, just squash multiple commits into one.
   
   When I squash multiple commits locally and force update my branch, https://github.com/apache/incubator-pinot/pull/6670 can not get updated, so close https://github.com/apache/incubator-pinot/pull/6670 and open a new one
   
   Sorry for the inconvenience
   
   ## Description
   Add a description of your PR here.
   A good description should include pointers to an issue or design document, etc.
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release.
   
   If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text
   
   ## Documentation
   If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] mqliang commented on a change in pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/combine/BaseCombineOperator.java
##########
@@ -52,77 +52,102 @@
   protected final ExecutorService _executorService;
   protected final long _endTimeMs;
 
+  protected final int _numOperators;
+  protected int _numThreads;
+  // Use a Phaser to ensure all the Futures are done (not scheduled, finished or interrupted) before the main thread
+  // returns. We need to ensure this because the main thread holds the reference to the segments. If a segment is
+  // deleted/refreshed, the segment will be released after the main thread returns, which would lead to undefined
+  // behavior (even JVM crash) when processing queries against it.
+  protected final Phaser _phaser = new Phaser(1);
+  protected final Future[] _futures;
+  // Use a _blockingQueue to store the per-segment result
+  private final BlockingQueue<IntermediateResultsBlock> _blockingQueue;
+
   public BaseCombineOperator(List<Operator> operators, QueryContext queryContext, ExecutorService executorService,
       long endTimeMs) {
     _operators = operators;
     _queryContext = queryContext;
     _executorService = executorService;
     _endTimeMs = endTimeMs;
+    _numOperators = _operators.size();
+    _numThreads = CombineOperatorUtils.getNumThreadsForQuery(_numOperators);

Review comment:
       I add a new constructor at L78, which can accept a `numThreads` parameter from subclass. GroupByCombineOperator and GroupByOrderByCombineOperator are now using the new constructor 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/combine/BaseCombineOperator.java
##########
@@ -52,77 +52,102 @@
   protected final ExecutorService _executorService;
   protected final long _endTimeMs;
 
+  protected final int _numOperators;
+  protected int _numThreads;
+  // Use a Phaser to ensure all the Futures are done (not scheduled, finished or interrupted) before the main thread
+  // returns. We need to ensure this because the main thread holds the reference to the segments. If a segment is
+  // deleted/refreshed, the segment will be released after the main thread returns, which would lead to undefined
+  // behavior (even JVM crash) when processing queries against it.
+  protected final Phaser _phaser = new Phaser(1);
+  protected final Future[] _futures;
+  // Use a _blockingQueue to store the per-segment result
+  private final BlockingQueue<IntermediateResultsBlock> _blockingQueue;
+
   public BaseCombineOperator(List<Operator> operators, QueryContext queryContext, ExecutorService executorService,
       long endTimeMs) {
     _operators = operators;
     _queryContext = queryContext;
     _executorService = executorService;
     _endTimeMs = endTimeMs;
+    _numOperators = _operators.size();
+    _numThreads = CombineOperatorUtils.getNumThreadsForQuery(_numOperators);

Review comment:
       We should try to unify this behavior of computing the number of threads (essentially tasks that we submit to the pool) in the outer loop.
   
   In the existing code, BaseCombineOperator and it's subclasses derive numThreads from numOperators using CombineOperatorUtils.getNumThreadsForQuery
   
   However, GroupByCombineOperator and GroupByOrderByCombineOperator don't derive numThreads from numOperators. They simply use numOperators in the outer loop. So, as part of the current refactoring to make all combine operators extend BaseCombineOperator, we should stick to the existing behavior. I think numThreads has to be passed to the base class from the subclasses
   
   Later on, we can see if the numThreads computation can be unified in a separate 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.

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



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


[GitHub] [incubator-pinot] mqliang commented on a change in pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/combine/GroupByOrderByCombineOperator.java
##########
@@ -39,165 +38,153 @@
 import org.apache.pinot.core.data.table.Key;
 import org.apache.pinot.core.data.table.Record;
 import org.apache.pinot.core.data.table.UnboundedConcurrentIndexedTable;
-import org.apache.pinot.core.operator.BaseOperator;
 import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
 import org.apache.pinot.core.query.aggregation.function.AggregationFunction;
 import org.apache.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
 import org.apache.pinot.core.query.aggregation.groupby.GroupKeyGenerator;
 import org.apache.pinot.core.query.exception.EarlyTerminationException;
 import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.core.util.GroupByUtils;
-import org.apache.pinot.core.util.trace.TraceRunnable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
 /**
  * Combine operator for aggregation group-by queries with SQL semantic.
- * TODO:
- *   - Use CombineOperatorUtils.getNumThreadsForQuery() to get the parallelism of the query instead of using all threads
- *   - Try to extend BaseCombineOperator to reduce duplicate code
+ * TODO: Use CombineOperatorUtils.getNumThreadsForQuery() to get the parallelism of the query instead of using
+ *   all threads
  */
 @SuppressWarnings("rawtypes")
-public class GroupByOrderByCombineOperator extends BaseOperator<IntermediateResultsBlock> {
+public class GroupByOrderByCombineOperator extends BaseCombineOperator {
+  public static final int MAX_TRIM_THRESHOLD = 1_000_000_000;
   private static final Logger LOGGER = LoggerFactory.getLogger(GroupByOrderByCombineOperator.class);
   private static final String OPERATOR_NAME = "GroupByOrderByCombineOperator";
-  public static final int MAX_TRIM_THRESHOLD = 1_000_000_000;
-
-  private final List<Operator> _operators;
-  private final QueryContext _queryContext;
-  private final ExecutorService _executorService;
-  private final long _endTimeMs;
   private final int _trimSize;
   private final int _trimThreshold;
   private final Lock _initLock;
+  private final int _numAggregationFunctions;
+  private final int _numGroupByExpressions;
+  private final int _numColumns;
+  private final ConcurrentLinkedQueue<ProcessingException> _mergedProcessingExceptions = new ConcurrentLinkedQueue<>();
+  // We use a CountDownLatch to track if all Futures are finished by the query timeout, and cancel the unfinished
+  // _futures (try to interrupt the execution if it already started).
+  // Besides the CountDownLatch, we also use a Phaser to ensure all the Futures are done (not scheduled, finished or
+  // interrupted) before the main thread returns. We need to ensure no execution left before the main thread returning
+  // because the main thread holds the reference to the segments, and if the segments are deleted/refreshed, the
+  // segments can be released after the main thread returns, which would lead to undefined behavior (even JVM crash)
+  // when executing queries against them.
+  private final CountDownLatch _operatorLatch;
   private DataSchema _dataSchema;
   private ConcurrentIndexedTable _indexedTable;
 
   public GroupByOrderByCombineOperator(List<Operator> operators, QueryContext queryContext,
       ExecutorService executorService, long endTimeMs, int trimThreshold) {
-    _operators = operators;
-    _queryContext = queryContext;
-    _executorService = executorService;
-    _endTimeMs = endTimeMs;
+    super(operators, queryContext, executorService, endTimeMs, operators.size());

Review comment:
       @siddharthteotia GroupByOrderByCombineOperator explicitly specify numberOfThread as numOfOperator 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.

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



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


[GitHub] [incubator-pinot] siddharthteotia merged pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged pull request #6678:
URL: https://github.com/apache/incubator-pinot/pull/6678


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] mqliang closed pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

Posted by GitBox <gi...@apache.org>.
mqliang closed pull request #6678:
URL: https://github.com/apache/incubator-pinot/pull/6678


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/combine/BaseCombineOperator.java
##########
@@ -52,77 +52,102 @@
   protected final ExecutorService _executorService;
   protected final long _endTimeMs;
 
+  protected final int _numOperators;
+  protected int _numThreads;
+  // Use a Phaser to ensure all the Futures are done (not scheduled, finished or interrupted) before the main thread
+  // returns. We need to ensure this because the main thread holds the reference to the segments. If a segment is
+  // deleted/refreshed, the segment will be released after the main thread returns, which would lead to undefined
+  // behavior (even JVM crash) when processing queries against it.
+  protected final Phaser _phaser = new Phaser(1);
+  protected final Future[] _futures;
+  // Use a _blockingQueue to store the per-segment result
+  private final BlockingQueue<IntermediateResultsBlock> _blockingQueue;
+
   public BaseCombineOperator(List<Operator> operators, QueryContext queryContext, ExecutorService executorService,
       long endTimeMs) {
     _operators = operators;
     _queryContext = queryContext;
     _executorService = executorService;
     _endTimeMs = endTimeMs;
+    _numOperators = _operators.size();
+    _numThreads = CombineOperatorUtils.getNumThreadsForQuery(_numOperators);

Review comment:
       The 2nd constructor may not be necessary as each subclass can pass this value to base class. It's ok for now since we have a couple of follow up PRs to finish the refactoring. We can re-evaluate then




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-pinot] mqliang commented on a change in pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/combine/GroupByCombineOperator.java
##########
@@ -63,24 +59,101 @@
   // in each segment. We still put a limit across segments to protect cases where data is very skewed across different
   // segments.
   private static final int INTER_SEGMENT_NUM_GROUPS_LIMIT_FACTOR = 2;
-
-  private final List<Operator> _operators;
-  private final QueryContext _queryContext;
-  private final ExecutorService _executorService;
-  private final long _endTimeMs;
   // Limit on number of groups stored, beyond which no new group will be created
   private final int _innerSegmentNumGroupsLimit;
   private final int _interSegmentNumGroupsLimit;
 
+  private final ConcurrentHashMap<String, Object[]> _resultsMap = new ConcurrentHashMap<>();
+  private final AtomicInteger _numGroups = new AtomicInteger();
+  private final ConcurrentLinkedQueue<ProcessingException> _mergedProcessingExceptions = new ConcurrentLinkedQueue<>();
+  private final AggregationFunction[] _aggregationFunctions;
+  private final int _numAggregationFunctions;
+  // We use a CountDownLatch to track if all Futures are finished by the query timeout, and cancel the unfinished
+  // _futures (try to interrupt the execution if it already started).
+  // Besides the CountDownLatch, we also use a Phaser to ensure all the Futures are done (not scheduled, finished or
+  // interrupted) before the main thread returns. We need to ensure no execution left before the main thread returning
+  // because the main thread holds the reference to the segments, and if the segments are deleted/refreshed, the
+  // segments can be released after the main thread returns, which would lead to undefined behavior (even JVM crash)
+  // when executing queries against them.
+  private final CountDownLatch _operatorLatch;
+
   public GroupByCombineOperator(List<Operator> operators, QueryContext queryContext, ExecutorService executorService,
       long endTimeMs, int innerSegmentNumGroupsLimit) {
-    _operators = operators;
-    _queryContext = queryContext;
-    _executorService = executorService;
-    _endTimeMs = endTimeMs;
+    super(operators, queryContext, executorService, endTimeMs, operators.size());

Review comment:
       @siddharthteotia GroupByCombineOperator explicitly specify numberOfThread as numOfOperator 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.

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



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


[GitHub] [incubator-pinot] codecov-io commented on pull request #6678: Extends GroupByCombineOperator and GroupByOrderByCombineOperator from BaseCombineOperator

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #6678:
URL: https://github.com/apache/incubator-pinot/pull/6678#issuecomment-797730137


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=h1) Report
   > Merging [#6678](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=desc) (4d64627) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) (1beaab5) will **decrease** coverage by `0.57%`.
   > The diff coverage is `62.64%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6678/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #6678      +/-   ##
   ==========================================
   - Coverage   66.44%   65.87%   -0.58%     
   ==========================================
     Files        1075     1387     +312     
     Lines       54773    67487   +12714     
     Branches     8168     9789    +1621     
   ==========================================
   + Hits        36396    44456    +8060     
   - Misses      15700    19876    +4176     
   - Partials     2677     3155     +478     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `65.87% <62.64%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <0.00%> (-79.32%)` | :arrow_down: |
   | [...pinot/broker/api/resources/PinotClientRequest.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdENsaWVudFJlcXVlc3QuamF2YQ==) | `0.00% <0.00%> (-27.28%)` | :arrow_down: |
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `71.42% <ø> (-28.58%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <0.00%> (-32.71%)` | :arrow_down: |
   | [...ker/routing/instanceselector/InstanceSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9pbnN0YW5jZXNlbGVjdG9yL0luc3RhbmNlU2VsZWN0b3IuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `66.66% <ø> (+9.52%)` | :arrow_up: |
   | [...n/java/org/apache/pinot/client/BrokerResponse.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Jyb2tlclJlc3BvbnNlLmphdmE=) | `100.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `35.55% <ø> (-13.29%)` | :arrow_down: |
   | [...org/apache/pinot/client/DynamicBrokerSelector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0R5bmFtaWNCcm9rZXJTZWxlY3Rvci5qYXZh) | `82.85% <ø> (+10.12%)` | :arrow_up: |
   | [...n/java/org/apache/pinot/client/ExecutionStats.java](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0V4ZWN1dGlvblN0YXRzLmphdmE=) | `68.88% <ø> (ø)` | |
   | ... and [1271 more](https://codecov.io/gh/apache/incubator-pinot/pull/6678/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=footer). Last update [c982721...4d64627](https://codecov.io/gh/apache/incubator-pinot/pull/6678?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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