sumitagrawl commented on code in PR #7517:
URL: https://github.com/apache/ozone/pull/7517#discussion_r1875889599
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java:
##########
@@ -145,7 +149,12 @@ public void triggerContainerHealthCheck() {
Time.monotonicNow() - start, existingCount);
checkAndProcessContainers(unhealthyContainerStateStatsMap, currentTime);
+ taskStatusCounter.updateCounter(ContainerHealthTask.class.getName(),
true);
Review Comment:
can use getTaskName() to be consistent.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.api.types.ReconTaskStatusStat;
+
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class contains definitions and implementation of Recon Task Status
counters
+ * For each task we maintain a count of the successes and the failures.
+ * This count is stored for a configurable
+ * {@link
org.apache.hadoop.ozone.recon.ReconServerConfigKeys#OZONE_RECON_TASK_STATUS_STORAGE_DURATION}
+ * which defaults to 30 minutes.
+ * Each task is mapped to a {@link Pair} of <code>{no. of successful runs, no.
of failed runs}</code>
+ */
+public class ReconTaskStatusCounter {
+ // Stores an instance of this class to maintain state across calls
+ private static ReconTaskStatusCounter instance;
+ // Stores the configurable timeout duration i.e. the TTL of the counts
+ private final long timeoutDuration;
+
+ // Task name is mapped from the enum to a Pair of <count of successful runs,
count of failed runs>
+ private static final Map<String, ReconTaskStatusStat> TASK_STATUS_COUNTER =
new HashMap<>();
Review Comment:
can this be changed to non-static, no use case of being static as it store
dynamic data.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconScmTask.java:
##########
@@ -40,8 +46,8 @@ protected ReconScmTask(ReconTaskStatusDao reconTaskStatusDao)
{
private void register() {
String taskName = getTaskName();
if (!reconTaskStatusDao.existsById(taskName)) {
- ReconTaskStatus reconTaskStatusRecord = new ReconTaskStatus(
- taskName, 0L, 0L);
+ reconTaskStatusRecord = new ReconTaskStatus(
+ taskName, 0L, 0L, 0, 0);
Review Comment:
Already we have. org.hadoop.ozone.recon.schema.tables.pojos.ReconTaskStatus
used, as we have created new set of classes, can we check existing logic
implementation and combine both as seems we are capturing extra information
only.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.api.types.ReconTaskStatusStat;
+
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class contains definitions and implementation of Recon Task Status
counters
+ * For each task we maintain a count of the successes and the failures.
+ * This count is stored for a configurable
+ * {@link
org.apache.hadoop.ozone.recon.ReconServerConfigKeys#OZONE_RECON_TASK_STATUS_STORAGE_DURATION}
+ * which defaults to 30 minutes.
+ * Each task is mapped to a {@link Pair} of <code>{no. of successful runs, no.
of failed runs}</code>
+ */
+public class ReconTaskStatusCounter {
+ // Stores an instance of this class to maintain state across calls
+ private static ReconTaskStatusCounter instance;
Review Comment:
We are init this in ReconControllerModule, then this need not be static.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.api.types.ReconTaskStatusStat;
+
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class contains definitions and implementation of Recon Task Status
counters
+ * For each task we maintain a count of the successes and the failures.
+ * This count is stored for a configurable
+ * {@link
org.apache.hadoop.ozone.recon.ReconServerConfigKeys#OZONE_RECON_TASK_STATUS_STORAGE_DURATION}
+ * which defaults to 30 minutes.
+ * Each task is mapped to a {@link Pair} of <code>{no. of successful runs, no.
of failed runs}</code>
+ */
+public class ReconTaskStatusCounter {
+ // Stores an instance of this class to maintain state across calls
+ private static ReconTaskStatusCounter instance;
+ // Stores the configurable timeout duration i.e. the TTL of the counts
+ private final long timeoutDuration;
+
+ // Task name is mapped from the enum to a Pair of <count of successful runs,
count of failed runs>
+ private static final Map<String, ReconTaskStatusStat> TASK_STATUS_COUNTER =
new HashMap<>();
+
+ public ReconTaskStatusCounter() {
+ OzoneConfiguration conf = new OzoneConfiguration();
+ timeoutDuration = conf.getTimeDuration(
+ OZONE_RECON_TASK_STATUS_STORAGE_DURATION,
+ OZONE_RECON_TASK_STATUS_STORAGE_DURATION_DEFAULT,
+ TimeUnit.MILLISECONDS
+ );
+ }
+
+ /**
+ * Checks if the duration of the counters exceeded
+ * the configured {@link org.apache.hadoop.ozone.recon.ReconServerConfigKeys
+ * OZONE_RECON_TASK_STATUS_STORAGE_DURATION} duration.
+ * Default duration/TTL of the counter is 30 minutes
+ * In case the count data TTL is reached, reinitialize the instance to reset
the data, else do nothing
+ */
+ private void checkCountDataExpiry(String taskName) {
+ ReconTaskStatusStat taskStat = TASK_STATUS_COUNTER.get(taskName);
+ //Since initially the task list is empty, each task will get initialized
at different times
+ if ((System.currentTimeMillis() - taskStat.getInitializationTime()) >
timeoutDuration) {
Review Comment:
no null check if taskStat is present or not, else this map needs to be
initialized during constructor OR to be ensured that this is never called in
get till fully initialized.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusCounter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.recon.metrics;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.api.types.ReconTaskStatusStat;
+
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class contains definitions and implementation of Recon Task Status
counters
+ * For each task we maintain a count of the successes and the failures.
+ * This count is stored for a configurable
+ * {@link
org.apache.hadoop.ozone.recon.ReconServerConfigKeys#OZONE_RECON_TASK_STATUS_STORAGE_DURATION}
+ * which defaults to 30 minutes.
+ * Each task is mapped to a {@link Pair} of <code>{no. of successful runs, no.
of failed runs}</code>
+ */
+public class ReconTaskStatusCounter {
+ // Stores an instance of this class to maintain state across calls
+ private static ReconTaskStatusCounter instance;
+ // Stores the configurable timeout duration i.e. the TTL of the counts
+ private final long timeoutDuration;
+
+ // Task name is mapped from the enum to a Pair of <count of successful runs,
count of failed runs>
+ private static final Map<String, ReconTaskStatusStat> TASK_STATUS_COUNTER =
new HashMap<>();
Review Comment:
TASK_STATUS_COUNTER need to be threadsafe, as being updated as well as
accessed during get.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconScmTask.java:
##########
@@ -32,6 +35,9 @@ public abstract class ReconScmTask {
private Thread taskThread;
private ReconTaskStatusDao reconTaskStatusDao;
private volatile boolean running;
+ private volatile ReconTaskStatus reconTaskStatusRecord;
Review Comment:
any specific reason to be volatile? can be more thread-safe ? if changing
object, need be handling null while access.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/PipelineSyncTask.java:
##########
@@ -55,6 +55,7 @@ public class PipelineSyncTask extends ReconScmTask {
private ReadWriteLock lock = new ReentrantReadWriteLock(true);
private final long interval;
+ private final ReconTaskStatusCounter taskStatusCounter;
Review Comment:
this is already present in base class, do not need as another member, but
can be access via base method to access and call further method.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java:
##########
@@ -84,10 +89,10 @@ public void registerTask(ReconOmTask task) {
// Store Task in Task failure tracker.
taskFailureCounter.put(taskName, new AtomicInteger(0));
// Create DB record for the task.
- ReconTaskStatus reconTaskStatusRecord = new ReconTaskStatus(taskName,
- 0L, 0L);
+ taskStatusMap.put(taskName, new ReconTaskStatus(
Review Comment:
already taskStatusMap is present as part of ReconTaskStatusCounter, seems
need use that only and to be initialized at same point of time to be in sync
with DB
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java:
##########
@@ -173,21 +179,27 @@ public synchronized void reInitializeTasks(
for (Map.Entry<String, ReconOmTask> taskEntry :
reconOmTasks.entrySet()) {
ReconOmTask task = taskEntry.getValue();
+ taskStatusMap.get(task.getTaskName()).setCurrentTaskRunStatus(1);
tasks.add(() -> task.reprocess(omMetadataManager));
}
List<Future<Pair<String, Boolean>>> results =
executorService.invokeAll(tasks);
for (Future<Pair<String, Boolean>> f : results) {
String taskName = f.get().getLeft();
+ ReconTaskStatus reconTaskStatusRecord = taskStatusMap.get(taskName);
+
reconTaskStatusRecord.setLastUpdatedSeqNumber(omMetadataManager.getLastSequenceNumberFromDB());
+
reconTaskStatusRecord.setLastUpdatedTimestamp(System.currentTimeMillis());
if (!f.get().getRight()) {
LOG.info("Init failed for task {}.", taskName);
+ reconTaskStatusRecord.setLastTaskRunStatus(-1);
+ taskStatusCounter.updateCounter(taskName, false);
} else {
//store the timestamp for the task
- ReconTaskStatus reconTaskStatusRecord = new ReconTaskStatus(taskName,
- System.currentTimeMillis(),
- omMetadataManager.getLastSequenceNumberFromDB());
- reconTaskStatusDao.update(reconTaskStatusRecord);
+ reconTaskStatusRecord.setLastTaskRunStatus(0);
+ taskStatusCounter.updateCounter(taskName, true);
}
+ reconTaskStatusRecord.setCurrentTaskRunStatus(0);
+ reconTaskStatusDao.update(reconTaskStatusRecord);
Review Comment:
Its not clear how are we in sync with DB and memory task map? during
startup, how are we reverse sync? and is it required? Is should cover startup
case, reinit case
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconTaskStatusMetrics.java:
##########
@@ -55,6 +55,14 @@ public class ReconTaskStatusMetrics implements MetricsSource
{
Interns.info("lastUpdatedSeqNumber",
"Last updated sequence number of corresponding Recon Task");
+ private static final MetricsInfo RECORD_INFO_LAST_TASK_RUN_STATUS =
Review Comment:
unable to find any usecase of 2 new MetricsInfo member variable
--
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]