kfaraz commented on code in PR #16829:
URL: https://github.com/apache/druid/pull/16829#discussion_r1702603291


##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java:
##########
@@ -45,7 +45,7 @@ public class NoopTask extends AbstractTask implements 
PendingSegmentAllocatingTa
   private static final int DEFAULT_RUN_TIME = 2500;
 
   @JsonIgnore
-  private final long runTime;
+  protected final long runTime;

Review Comment:
   This change should not be needed. Use getter instead.



##########
docs/operations/metrics.md:
##########
@@ -297,6 +297,8 @@ If the JVM does not support CPU time measurement for the 
current thread, `ingest
 |`worker/taskSlot/used/count`|Number of busy task slots on the reporting 
worker per emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.| `category`, 
`workerVersion`|Varies|
 |`worker/task/assigned/count`|Number of tasks assigned to an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
 |`worker/task/completed/count`|Number of tasks completed by an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
+|`indexer/task/failed/count`|Number of tasks failed on an indexer per emission 
period. This metric is only available if the `WorkerTaskCountStatsMonitor` 
module is included.|`dataSource`|Varies|

Review Comment:
   ```suggestion
   |`indexer/task/failed/count`|Number of tasks that failed on an indexer 
during the emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
   ```



##########
indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java:
##########
@@ -455,6 +455,19 @@ private NoopTask createNoopTask(String id, String 
dataSource)
     return new NoopTask(id, null, dataSource, 100, 0, 
ImmutableMap.of(Tasks.PRIORITY_KEY, 0));
   }
 
+  private NoopTask createNoopFailingTask(String id, String dataSource)
+  {
+    return new NoopTask(id, null, dataSource, 100, 0, 
ImmutableMap.of(Tasks.PRIORITY_KEY, 0))
+    {
+      @Override
+      public TaskStatus runTask(TaskToolbox toolbox) throws Exception
+      {
+        Thread.sleep(runTime);

Review Comment:
   ```suggestion
           Thread.sleep(getRunTime());
   ```



##########
docs/operations/metrics.md:
##########
@@ -297,6 +297,8 @@ If the JVM does not support CPU time measurement for the 
current thread, `ingest
 |`worker/taskSlot/used/count`|Number of busy task slots on the reporting 
worker per emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.| `category`, 
`workerVersion`|Varies|
 |`worker/task/assigned/count`|Number of tasks assigned to an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
 |`worker/task/completed/count`|Number of tasks completed by an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
+|`indexer/task/failed/count`|Number of tasks failed on an indexer per emission 
period. This metric is only available if the `WorkerTaskCountStatsMonitor` 
module is included.|`dataSource`|Varies|
+|`indexer/task/success/count`|Number of tasks succeeded on an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|

Review Comment:
   ```suggestion
   |`indexer/task/success/count`|Number of tasks that succeeded on an indexer 
during the emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
   ```



##########
docs/operations/metrics.md:
##########
@@ -297,6 +297,8 @@ If the JVM does not support CPU time measurement for the 
current thread, `ingest
 |`worker/taskSlot/used/count`|Number of busy task slots on the reporting 
worker per emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.| `category`, 
`workerVersion`|Varies|
 |`worker/task/assigned/count`|Number of tasks assigned to an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
 |`worker/task/completed/count`|Number of tasks completed by an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|
+|`indexer/task/failed/count`|Number of tasks failed on an indexer per emission 
period. This metric is only available if the `WorkerTaskCountStatsMonitor` 
module is included.|`dataSource`|Varies|
+|`indexer/task/success/count`|Number of tasks succeeded on an indexer per 
emission period. This metric is only available if the 
`WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies|

Review Comment:
   Why not name these metrics in the same style as 
`worker/task/completed/count` i.e. `worker/task/success/count` and 
`worker/task/failed/count`? 
   
   They are being emitted by the same service using the same mechanism. I think 
the metric names should align to the existing scheme.



##########
server/src/main/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitor.java:
##########
@@ -72,6 +72,8 @@ public boolean doMonitor(ServiceEmitter emitter)
       emit(emitter, "worker/task/running/count", 
indexerStatsProvider.getWorkerRunningTasks());
       emit(emitter, "worker/task/assigned/count", 
indexerStatsProvider.getWorkerAssignedTasks());
       emit(emitter, "worker/task/completed/count", 
indexerStatsProvider.getWorkerCompletedTasks());
+      emit(emitter, "indexer/task/failed/count", 
indexerStatsProvider.getWorkerFailedTasks());
+      emit(emitter, "indexer/task/success/count", 
indexerStatsProvider.getWorkerSuccessfulTasks());

Review Comment:
   Metrics should be renamed to use prefix `worker/task/...`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to