devmadhuu commented on code in PR #7517:
URL: https://github.com/apache/ozone/pull/7517#discussion_r1867573587
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,127 @@
+package org.apache.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION;
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION_DEFAULT;
+
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class ReconTaskStatusCounter {
+ private static ReconTaskStatusCounter instance;
+ private final long timeoutDuration;
+
+ public enum ReconTasks {
+ ContainerHealthTask,
+ ContainerKeyMapperTask,
+ ContainerSizeCountTask,
+ FileSizeCountTask,
+ NSSummaryTask,
+ OmDeltaRequest,
+ OmTableInsightTask,
+ OmSnapshotRequest,
+ PipelineSyncTask,
+ ReconScmTask
+ }
+
+ static long initializationTime = 0L;
+
+ static Map<ReconTasks, Pair<Integer, Integer>> taskStatusCounter= new
EnumMap<>(ReconTasks.class);
Review Comment:
Pls add a comment what all values this `Pair` represents ?
##########
hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconTaskSchemaDefinition.java:
##########
@@ -64,6 +64,7 @@ private void createReconTaskStatusTable(Connection conn) {
.column("task_name", SQLDataType.VARCHAR(766).nullable(false))
.column("last_updated_timestamp", SQLDataType.BIGINT)
.column("last_updated_seq_number", SQLDataType.BIGINT)
+ .column("last_task_successful", SQLDataType.BIT)
Review Comment:
We need one more status column to track `current_run_task_status`. If
currently task is running then it can be updated with value `1`, else `0`.
Currently no way to know if task is running or finished.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/ReconTaskStatusCountResponse.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+
+/**
+ * Class to represent the API response structure of task status metrics
Review Comment:
```suggestion
* Class to represent the API response structure of task status statistics.
```
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/ReconTaskStatusCountResponse.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+
+/**
+ * Class to represent the API response structure of task status metrics
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ReconTaskStatusCountResponse {
+
+ // The name of the task for which we are getting status
+ @JsonProperty("taskName")
+ private String taskName;
+
+ // The number of successes associated with the task
+ @JsonProperty("successes")
+ private int successes;
+
+ // The number of failures associated with the task
+ @JsonProperty("failures")
+ private int failures;
Review Comment:
```suggestion
private int failureCount;
```
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java:
##########
@@ -85,7 +88,7 @@ public void registerTask(ReconOmTask task) {
taskFailureCounter.put(taskName, new AtomicInteger(0));
// Create DB record for the task.
ReconTaskStatus reconTaskStatusRecord = new ReconTaskStatus(taskName,
- 0L, 0L);
+ 0L, 0L, null);
Review Comment:
```suggestion
0L, 0L, 0);
```
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TaskStatusService.java:
##########
@@ -50,4 +69,37 @@ public Response getTaskTimes() {
List<ReconTaskStatus> resultSet = reconTaskStatusDao.findAll();
return Response.ok(resultSet).build();
}
+
+ @GET
+ @Path("metrics")
+ public Response getTaskMetrics() {
+ Map<ReconTaskStatusCounter.ReconTasks, Pair<Integer, Integer>>
tasksPairMap = taskStatusCounter.getTaskCounts();
+ List<ReconTaskStatusCountResponse> tasks = new ArrayList<>();
+ for (Map.Entry<ReconTaskStatusCounter.ReconTasks, Pair<Integer, Integer>>
entry: tasksPairMap.entrySet()) {
+ tasks.add(new ReconTaskStatusCountResponse(
+ entry.getKey().name(),
+ entry.getValue().getLeft(),
+ entry.getValue().getRight()
+ ));
+ }
+ return Response.ok(new ReconAllTasksCountResponse(tasks)).build();
+ }
+
+ @GET
+ @Path("metrics/{taskName}")
+ public Response getTaskMetrics(
Review Comment:
Also the method name is not meaningful. Pls refer previous comment.
##########
hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconTaskSchemaDefinition.java:
##########
@@ -64,6 +64,7 @@ private void createReconTaskStatusTable(Connection conn) {
.column("task_name", SQLDataType.VARCHAR(766).nullable(false))
.column("last_updated_timestamp", SQLDataType.BIGINT)
.column("last_updated_seq_number", SQLDataType.BIGINT)
+ .column("last_task_successful", SQLDataType.BIT)
Review Comment:
```suggestion
.column("last_task_run_status", SQLDataType.BIT)
```
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java:
##########
@@ -186,6 +186,11 @@ public final class ReconServerConfigKeys {
public static final int
OZONE_RECON_SCM_CLIENT_FAILOVER_MAX_RETRY_DEFAULT = 3;
+ public static final String OZONE_RECON_TASK_STATUS_STORAGE_DURATION =
+ "ozone.recon.task.status.storage.duration";
Review Comment:
If this is new config, then needs to be updated in `ozone-default.xml `as
well.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TaskStatusService.java:
##########
@@ -50,4 +69,37 @@ public Response getTaskTimes() {
List<ReconTaskStatus> resultSet = reconTaskStatusDao.findAll();
return Response.ok(resultSet).build();
}
+
+ @GET
+ @Path("metrics")
+ public Response getTaskMetrics() {
+ Map<ReconTaskStatusCounter.ReconTasks, Pair<Integer, Integer>>
tasksPairMap = taskStatusCounter.getTaskCounts();
+ List<ReconTaskStatusCountResponse> tasks = new ArrayList<>();
+ for (Map.Entry<ReconTaskStatusCounter.ReconTasks, Pair<Integer, Integer>>
entry: tasksPairMap.entrySet()) {
+ tasks.add(new ReconTaskStatusCountResponse(
+ entry.getKey().name(),
+ entry.getValue().getLeft(),
+ entry.getValue().getRight()
+ ));
+ }
+ return Response.ok(new ReconAllTasksCountResponse(tasks)).build();
+ }
+
+ @GET
+ @Path("metrics/{taskName}")
+ public Response getTaskMetrics(
Review Comment:
why we need this ?
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,127 @@
+package org.apache.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION;
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION_DEFAULT;
+
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class ReconTaskStatusCounter {
Review Comment:
Add class comment.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TaskStatusService.java:
##########
@@ -50,4 +69,37 @@ public Response getTaskTimes() {
List<ReconTaskStatus> resultSet = reconTaskStatusDao.findAll();
return Response.ok(resultSet).build();
}
+
+ @GET
+ @Path("metrics")
Review Comment:
As far as I understand, this endpoint is to expose the statistics of task
which means how many times a task was failure or success in last X units of
configured time value. So give some meaningful name of method as well as
endPoint. This endpoint is not exposing metrics.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/ReconTaskStatusCountResponse.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+
+/**
+ * Class to represent the API response structure of task status metrics
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ReconTaskStatusCountResponse {
+
+ // The name of the task for which we are getting status
+ @JsonProperty("taskName")
+ private String taskName;
+
+ // The number of successes associated with the task
+ @JsonProperty("successes")
+ private int successes;
Review Comment:
```suggestion
private int successCount;
```
##########
hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconTaskSchemaDefinition.java:
##########
@@ -64,6 +64,7 @@ private void createReconTaskStatusTable(Connection conn) {
.column("task_name", SQLDataType.VARCHAR(766).nullable(false))
.column("last_updated_timestamp", SQLDataType.BIGINT)
.column("last_updated_seq_number", SQLDataType.BIGINT)
+ .column("last_task_successful", SQLDataType.BIT)
Review Comment:
Also add 2 these 2 new columns metrics in existing `ReconTaskStatusMetrics`
class as well so that they are exposed via `prometheus`
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,127 @@
+package org.apache.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION;
+import static
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_TASK_STATUS_STORAGE_DURATION_DEFAULT;
+
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class ReconTaskStatusCounter {
+ private static ReconTaskStatusCounter instance;
+ private final long timeoutDuration;
+
+ public enum ReconTasks {
+ ContainerHealthTask,
+ ContainerKeyMapperTask,
+ ContainerSizeCountTask,
+ FileSizeCountTask,
+ NSSummaryTask,
+ OmDeltaRequest,
+ OmTableInsightTask,
+ OmSnapshotRequest,
+ PipelineSyncTask,
+ ReconScmTask
+ }
+
+ static long initializationTime = 0L;
+
+ static Map<ReconTasks, Pair<Integer, Integer>> taskStatusCounter= new
EnumMap<>(ReconTasks.class);
+
+ public ReconTaskStatusCounter() {
+ OzoneConfiguration conf = new OzoneConfiguration();
+ timeoutDuration = conf.getTimeDuration(
+ OZONE_RECON_TASK_STATUS_STORAGE_DURATION,
+ OZONE_RECON_TASK_STATUS_STORAGE_DURATION_DEFAULT,
+ TimeUnit.MILLISECONDS
+ );
+
+ initializationTime = System.currentTimeMillis();
+ for (ReconTasks task: ReconTasks.values()) {
+ taskStatusCounter.put(task, Pair.of(0, 0));
+ }
+ }
+
+ /**
+ * Get an instance of <code>this</code> {@link ReconTaskStatusCounter} in
order to persist state
+ * of the task counters between multiple modules/packages
+ * @return an instance of current {@link ReconTaskStatusCounter}
+ */
+ public static ReconTaskStatusCounter getCurrentInstance() {
+ if (null == instance) {
+ instance = new ReconTaskStatusCounter();
+ }
+ return instance;
+ }
+
+ /**
+ * Update the counter's success/failure count based on the task class passed
+ * @param clazz An instance of {@link Class} of the task for which we want
to update the counter
+ * @param successful Whether the task was successful or not
+ */
+ public void updateCounter(Class<?> clazz, boolean successful) {
Review Comment:
Is this method needed ? Looks like not being referenced ?
##########
hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconTaskSchemaDefinition.java:
##########
@@ -64,6 +64,7 @@ private void createReconTaskStatusTable(Connection conn) {
.column("task_name", SQLDataType.VARCHAR(766).nullable(false))
.column("last_updated_timestamp", SQLDataType.BIGINT)
.column("last_updated_seq_number", SQLDataType.BIGINT)
+ .column("last_task_successful", SQLDataType.BIT)
Review Comment:
Also use as INTEGER datatype, it will be easy to integrate with existing
hadoop metrics.
--
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]