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 2022/12/27 09:12:42 UTC

[GitHub] [pinot] zhtaoxiang opened a new pull request, #10037: report table task type level execution metrics

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

   report table task type level execution metrics so that it will be easier to debug minion tasks at table task type level


-- 
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] snleee commented on a diff in pull request #10037: report table task type level execution metrics

Posted by GitBox <gi...@apache.org>.
snleee commented on code in PR #10037:
URL: https://github.com/apache/pinot/pull/10037#discussion_r1057996304


##########
pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java:
##########
@@ -87,55 +87,91 @@ public TaskResult run() {
               _minionMetrics
                   .addPhaseTiming(taskType, MinionQueryPhase.TASK_QUEUEING, jobDequeueTimeMs - jobInQueueTimeMs,
                       TimeUnit.MILLISECONDS);
+              String tableName = null;
               try {
                 // Set taskId in MDC so that one may config logger to route task logs to separate file.
                 MDC.put("taskId", _taskConfig.getId());
+                PinotTaskConfig pinotTaskConfig = PinotTaskConfig.fromHelixTaskConfig(_taskConfig);
+                tableName = pinotTaskConfig.getTableName();
                 _minionMetrics.addValueToGlobalGauge(MinionGauge.NUMBER_OF_TASKS, 1L);
+                if (tableName != null) {
+                  _minionMetrics
+                      .addPhaseTiming(tableName + "." + taskType, MinionQueryPhase.TASK_QUEUEING,
+                          jobDequeueTimeMs - jobInQueueTimeMs, TimeUnit.MILLISECONDS);
+                  _minionMetrics.addValueToTableGauge(tableName, MinionGauge.NUMBER_OF_TASKS, 1L);
+                }
                 MinionEventObservers.getInstance().addMinionEventObserver(_taskConfig.getId(), _eventObserver);
-                return runInternal();
+                return runInternal(pinotTaskConfig);
               } finally {
                 MinionEventObservers.getInstance().removeMinionEventObserver(_taskConfig.getId());
                 _minionMetrics.addValueToGlobalGauge(MinionGauge.NUMBER_OF_TASKS, -1L);
                 long executionTimeMs = System.currentTimeMillis() - jobDequeueTimeMs;
                 _minionMetrics
                     .addPhaseTiming(taskType, MinionQueryPhase.TASK_EXECUTION, executionTimeMs, TimeUnit.MILLISECONDS);
+                if (tableName != null) {
+                  _minionMetrics.addValueToTableGauge(tableName, MinionGauge.NUMBER_OF_TASKS, -1L);
+                  _minionMetrics
+                      .addPhaseTiming(tableName + "." + taskType, MinionQueryPhase.TASK_EXECUTION,
+                          executionTimeMs, TimeUnit.MILLISECONDS);
+                }
                 LOGGER.info("Task: {} completed in: {}ms", _taskConfig.getId(), executionTimeMs);
                 // Clear taskId from MDC to reset it.
                 MDC.remove("taskId");
               }
             }
 
-            private TaskResult runInternal() {
-              PinotTaskConfig pinotTaskConfig = PinotTaskConfig.fromHelixTaskConfig(_taskConfig);
+            private TaskResult runInternal(PinotTaskConfig pinotTaskConfig) {
               if (StringUtils.isBlank(pinotTaskConfig.getConfigs().get(MinionConstants.AUTH_TOKEN))) {
                 pinotTaskConfig.getConfigs().put(MinionConstants.AUTH_TOKEN,
                     AuthProviderUtils.toStaticToken(MinionContext.getInstance().getTaskAuthProvider()));
               }
 
+              String tableName = pinotTaskConfig.getTableName();
+
               _eventObserver.notifyTaskStart(pinotTaskConfig);
               _minionMetrics.addMeteredTableValue(taskType, MinionMeter.NUMBER_TASKS_EXECUTED, 1L);
+              if (tableName != null) {
+                _minionMetrics.addMeteredTableValue(tableName + "." + taskType,

Review Comment:
   I think that we have been using `tableName` as the key for the metrics. We probably need to provide an extra input `key` to the function in case they want some other dimensions. As we discussed, we need sth like the following:
   ```
     // metric key will be `<tableNameWithType>.<key>.<meter name>`
     public void addMeteredTableValue(final String tableName, final String key, final M meter, final long unitCount) {
     }
   ```
   
   We can add this function and start to use the new function for this PR. Or, we can at least file the issue and address it all together. 



##########
docker/images/pinot/etc/jmx_prometheus_javaagent/configs/minion.yml:
##########
@@ -1,4 +1,22 @@
 rules:
+- pattern: "\"org.apache.pinot.common.metrics\"<type=\"MinionMetrics\", name=\"pinot.minion.version.(\\w+)\"><>(\\w+)"

Review Comment:
   discussed offline.



-- 
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] zhtaoxiang commented on a diff in pull request #10037: report table task type level execution metrics

Posted by GitBox <gi...@apache.org>.
zhtaoxiang commented on code in PR #10037:
URL: https://github.com/apache/pinot/pull/10037#discussion_r1057546710


##########
docker/images/pinot/etc/jmx_prometheus_javaagent/configs/minion.yml:
##########
@@ -1,4 +1,22 @@
 rules:
+- pattern: "\"org.apache.pinot.common.metrics\"<type=\"MinionMetrics\", name=\"pinot.minion.version.(\\w+)\"><>(\\w+)"

Review Comment:
   move this rule here so that it will take effects before more generic ones.



-- 
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] snleee merged pull request #10037: report table task type level execution metrics

Posted by GitBox <gi...@apache.org>.
snleee merged PR #10037:
URL: https://github.com/apache/pinot/pull/10037


-- 
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 #10037: report table task type level execution metrics

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #10037:
URL: https://github.com/apache/pinot/pull/10037#issuecomment-1365765297

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10037?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 [#10037](https://codecov.io/gh/apache/pinot/pull/10037?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (150b557) into [master](https://codecov.io/gh/apache/pinot/commit/431b91802c10401264bac8a30b0da12cf6a43614?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (431b918) will **decrease** coverage by `56.82%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10037       +/-   ##
   =============================================
   - Coverage     70.42%   13.59%   -56.83%     
   + Complexity     5691      176     -5515     
   =============================================
     Files          1994     1939       -55     
     Lines        107508   105063     -2445     
     Branches      16340    16053      -287     
   =============================================
   - Hits          75709    14281    -61428     
   - Misses        26502    89655    +63153     
   + Partials       5297     1127     -4170     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `13.59% <0.00%> (-0.03%)` | :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/10037?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/core/minion/PinotTaskConfig.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9taW5pb24vUGlub3RUYXNrQ29uZmlnLmphdmE=) | `0.00% <0.00%> (-54.55%)` | :arrow_down: |
   | [.../pinot/minion/taskfactory/TaskFactoryRegistry.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vdGFza2ZhY3RvcnkvVGFza0ZhY3RvcnlSZWdpc3RyeS5qYXZh) | `0.00% <0.00%> (-79.42%)` | :arrow_down: |
   | [...src/main/java/org/apache/pinot/sql/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?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/spi/utils/LoopUtils.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvTG9vcFV0aWxzLmphdmE=) | `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/10037/diff?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/10037/diff?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: |
   | [...n/java/org/apache/pinot/core/data/table/Table.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?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: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/spi/trace/BaseRecording.java](https://codecov.io/gh/apache/pinot/pull/10037/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdHJhY2UvQmFzZVJlY29yZGluZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1566 more](https://codecov.io/gh/apache/pinot/pull/10037/diff?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] zhtaoxiang commented on a diff in pull request #10037: report table task type level execution metrics

Posted by GitBox <gi...@apache.org>.
zhtaoxiang commented on code in PR #10037:
URL: https://github.com/apache/pinot/pull/10037#discussion_r1058175957


##########
pinot-minion/src/main/java/org/apache/pinot/minion/taskfactory/TaskFactoryRegistry.java:
##########
@@ -87,55 +87,91 @@ public TaskResult run() {
               _minionMetrics
                   .addPhaseTiming(taskType, MinionQueryPhase.TASK_QUEUEING, jobDequeueTimeMs - jobInQueueTimeMs,
                       TimeUnit.MILLISECONDS);
+              String tableName = null;
               try {
                 // Set taskId in MDC so that one may config logger to route task logs to separate file.
                 MDC.put("taskId", _taskConfig.getId());
+                PinotTaskConfig pinotTaskConfig = PinotTaskConfig.fromHelixTaskConfig(_taskConfig);
+                tableName = pinotTaskConfig.getTableName();
                 _minionMetrics.addValueToGlobalGauge(MinionGauge.NUMBER_OF_TASKS, 1L);
+                if (tableName != null) {
+                  _minionMetrics
+                      .addPhaseTiming(tableName + "." + taskType, MinionQueryPhase.TASK_QUEUEING,
+                          jobDequeueTimeMs - jobInQueueTimeMs, TimeUnit.MILLISECONDS);
+                  _minionMetrics.addValueToTableGauge(tableName, MinionGauge.NUMBER_OF_TASKS, 1L);
+                }
                 MinionEventObservers.getInstance().addMinionEventObserver(_taskConfig.getId(), _eventObserver);
-                return runInternal();
+                return runInternal(pinotTaskConfig);
               } finally {
                 MinionEventObservers.getInstance().removeMinionEventObserver(_taskConfig.getId());
                 _minionMetrics.addValueToGlobalGauge(MinionGauge.NUMBER_OF_TASKS, -1L);
                 long executionTimeMs = System.currentTimeMillis() - jobDequeueTimeMs;
                 _minionMetrics
                     .addPhaseTiming(taskType, MinionQueryPhase.TASK_EXECUTION, executionTimeMs, TimeUnit.MILLISECONDS);
+                if (tableName != null) {
+                  _minionMetrics.addValueToTableGauge(tableName, MinionGauge.NUMBER_OF_TASKS, -1L);
+                  _minionMetrics
+                      .addPhaseTiming(tableName + "." + taskType, MinionQueryPhase.TASK_EXECUTION,
+                          executionTimeMs, TimeUnit.MILLISECONDS);
+                }
                 LOGGER.info("Task: {} completed in: {}ms", _taskConfig.getId(), executionTimeMs);
                 // Clear taskId from MDC to reset it.
                 MDC.remove("taskId");
               }
             }
 
-            private TaskResult runInternal() {
-              PinotTaskConfig pinotTaskConfig = PinotTaskConfig.fromHelixTaskConfig(_taskConfig);
+            private TaskResult runInternal(PinotTaskConfig pinotTaskConfig) {
               if (StringUtils.isBlank(pinotTaskConfig.getConfigs().get(MinionConstants.AUTH_TOKEN))) {
                 pinotTaskConfig.getConfigs().put(MinionConstants.AUTH_TOKEN,
                     AuthProviderUtils.toStaticToken(MinionContext.getInstance().getTaskAuthProvider()));
               }
 
+              String tableName = pinotTaskConfig.getTableName();
+
               _eventObserver.notifyTaskStart(pinotTaskConfig);
               _minionMetrics.addMeteredTableValue(taskType, MinionMeter.NUMBER_TASKS_EXECUTED, 1L);
+              if (tableName != null) {
+                _minionMetrics.addMeteredTableValue(tableName + "." + taskType,

Review Comment:
   added new 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