devmadhuu commented on code in PR #7835:
URL: https://github.com/apache/ozone/pull/7835#discussion_r1950249835
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -137,14 +150,11 @@ public Pair<String, Boolean> reprocess(OMMetadataManager
omMetadataManager) {
return new ImmutablePair<>(getTaskName(), false);
}
- LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask.");
- Instant end = Instant.now();
- long duration = Duration.between(start, end).toMillis();
- LOG.debug("It took me {} seconds to process {} keys.",
- (double) duration / 1000.0, omKeyCount);
+ LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask on {} keys.",
omKeyCount);
} catch (IOException ioEx) {
LOG.error("Unable to populate Container Key data in Recon DB. ",
ioEx);
+ metrics.incrTaskReprocessFailureCount();
Review Comment:
Pls check , we have multiple failure points for `reprocess` in the task
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -87,45 +96,49 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
long omKeyCount = 0;
-
+ metrics.incrTaskReprocessCount();
// In-memory maps for fast look up and batch write
// (container, key) -> count
Map<ContainerKeyPrefix, Integer> containerKeyMap = new HashMap<>();
// containerId -> key count
Map<Long, Long> containerKeyCountMap = new HashMap<>();
try {
LOG.debug("Starting a 'reprocess' run of ContainerKeyMapperTask.");
- Instant start = Instant.now();
// initialize new container DB
reconContainerMetadataManager
.reinitWithNewContainerDataFromOm(new HashMap<>());
// loop over both key table and file table
- for (BucketLayout layout : Arrays.asList(BucketLayout.LEGACY,
- BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
- // (HDDS-8580) Since "reprocess" iterate over the whole key table,
- // containerKeyMap needs to be incrementally flushed to DB based on
- // configured batch threshold.
- // containerKeyCountMap can be flushed at the end since the number
- // of containers in a cluster will not have significant memory
overhead.
- Table<String, OmKeyInfo> omKeyInfoTable =
- omMetadataManager.getKeyTable(layout);
- try (
- TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
- keyIter = omKeyInfoTable.iterator()) {
- while (keyIter.hasNext()) {
- Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
- OmKeyInfo omKeyInfo = kv.getValue();
- handleKeyReprocess(kv.getKey(), omKeyInfo, containerKeyMap,
- containerKeyCountMap);
- if (!checkAndCallFlushToDB(containerKeyMap)) {
- LOG.error("Unable to flush containerKey information to the DB");
- return new ImmutablePair<>(getTaskName(), false);
+ long startTime = Time.monotonicNow();
Review Comment:
Could this start time be moved before `reinitWithNewContainerDataFromOm` ?
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -249,31 +264,37 @@ public Pair<String, Boolean> process(OMUpdateEventBatch
events) {
} catch (IOException e) {
LOG.error("Unexpected exception while updating key data : {} ",
updatedKey, e);
+ metrics.incrTaskProcessFailureCount();
Review Comment:
What about runtime exceptions which are thrown back to caller of `process`
? Runtime error are most common in ReconOM task framework.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -193,6 +204,7 @@ public Collection<String> getTaskTables() {
@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+ metrics.incrTaskProcessCount();
Review Comment:
same comment as given for `reprocess`
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java:
##########
@@ -519,6 +523,7 @@ boolean innerGetAndApplyDeltaUpdatesFromOM(long
fromSequenceNumber,
}
}
}
+ metrics.updateDeltaUpdateWriteLatency(Time.monotonicNow() - startTime);
Review Comment:
We should capture separate metrics for converting the OM DB updates into
recon events and separate for writing/applying OM DB updates to Recon OM
snapshot DB.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -220,16 +232,19 @@ public Pair<String, Boolean> process(OMUpdateEventBatch
events) {
try {
switch (omdbUpdateEvent.getAction()) {
case PUT:
+ metrics.incrPutKeyEventCount();
handlePutOMKeyEvent(updatedKey, updatedKeyValue, containerKeyMap,
containerKeyCountMap, deletedKeyCountList);
break;
case DELETE:
+ metrics.incrDeleteKeyEventCount();
handleDeleteOMKeyEvent(updatedKey, containerKeyMap,
containerKeyCountMap, deletedKeyCountList);
break;
case UPDATE:
+ metrics.incrUpdateKeyEventCount();
Review Comment:
Metrics capture for all `PUT`, `DELETE` and `UPDATE` events should be moved
after the actual processing of event.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -296,12 +318,15 @@ private void writeToTheDB(Map<ContainerKeyPrefix,
Integer> containerKeyMap,
reconContainerMetadataManager
.batchDeleteContainerMapping(rdbBatchOperation, key);
} catch (IOException e) {
+ metrics.incrTaskWriteToDBFailureCount();
Review Comment:
We are incrementing the same write to db failure count metric for 3
different types of data write to DB, is this expected ?
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/metrics/ReconOmTaskMetrics.java:
##########
@@ -0,0 +1,104 @@
+package org.apache.hadoop.ozone.recon.metrics;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+/**
+ * This interface is to be implemented by the various metrics for different
tasks
+ */
+public abstract class ReconOmTaskMetrics implements MetricsSource {
+
+ private final MetricsRegistry registry = new MetricsRegistry("ReconOMTasks");
+ private final String taskName;
+ private final String source;
+
+ private @Metric MutableCounterLong reprocessCount;
+ private @Metric MutableCounterLong reprocessFailedCount;
+ private @Metric MutableRate reprocessLatency;
+
+ private @Metric MutableCounterLong processCount;
+ private @Metric MutableCounterLong processFailedCount;
+ private @Metric MutableRate processLatency;
+
+ protected ReconOmTaskMetrics(String taskName, String source) {
+ this.taskName = taskName;
+ this.source = source;
+ }
+
+ /**
+ * Update the number of reprocess() method calls by the task
+ */
+ public void incrTaskReprocessCount() {
Review Comment:
Shouldn't this metric be captured at the end of `reprocess` ?because
`reprocess` may fail, but we still incremented this metric ?
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -249,31 +264,37 @@ public Pair<String, Boolean> process(OMUpdateEventBatch
events) {
} catch (IOException e) {
LOG.error("Unexpected exception while updating key data : {} ",
updatedKey, e);
+ metrics.incrTaskProcessFailureCount();
return new ImmutablePair<>(getTaskName(), false);
}
}
+ metrics.updateTaskProcessLatency(Time.monotonicNow() - startTime);
+
try {
writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList);
} catch (IOException e) {
LOG.error("Unable to write Container Key Prefix data in Recon DB.", e);
+ metrics.incrTaskWriteToDBFailureCount();
return new ImmutablePair<>(getTaskName(), false);
}
- LOG.debug("{} successfully processed {} OM DB update event(s) in {}
milliseconds.",
- getTaskName(), eventCount, (System.currentTimeMillis() - startTime));
+ LOG.debug("{} successfully processed {} OM DB update event(s)",
getTaskName(), eventCount);
return new ImmutablePair<>(getTaskName(), true);
}
private void writeToTheDB(Map<ContainerKeyPrefix, Integer> containerKeyMap,
Map<Long, Long> containerKeyCountMap,
List<ContainerKeyPrefix> deletedContainerKeyList)
throws IOException {
+ metrics.incrTaskWriteToDBCount();
Review Comment:
IMO, should be moved at the end in finally.
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -87,45 +96,49 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
long omKeyCount = 0;
-
+ metrics.incrTaskReprocessCount();
// In-memory maps for fast look up and batch write
// (container, key) -> count
Map<ContainerKeyPrefix, Integer> containerKeyMap = new HashMap<>();
// containerId -> key count
Map<Long, Long> containerKeyCountMap = new HashMap<>();
try {
LOG.debug("Starting a 'reprocess' run of ContainerKeyMapperTask.");
- Instant start = Instant.now();
// initialize new container DB
reconContainerMetadataManager
.reinitWithNewContainerDataFromOm(new HashMap<>());
// loop over both key table and file table
- for (BucketLayout layout : Arrays.asList(BucketLayout.LEGACY,
- BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
- // (HDDS-8580) Since "reprocess" iterate over the whole key table,
- // containerKeyMap needs to be incrementally flushed to DB based on
- // configured batch threshold.
- // containerKeyCountMap can be flushed at the end since the number
- // of containers in a cluster will not have significant memory
overhead.
- Table<String, OmKeyInfo> omKeyInfoTable =
- omMetadataManager.getKeyTable(layout);
- try (
- TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
- keyIter = omKeyInfoTable.iterator()) {
- while (keyIter.hasNext()) {
- Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
- OmKeyInfo omKeyInfo = kv.getValue();
- handleKeyReprocess(kv.getKey(), omKeyInfo, containerKeyMap,
- containerKeyCountMap);
- if (!checkAndCallFlushToDB(containerKeyMap)) {
- LOG.error("Unable to flush containerKey information to the DB");
- return new ImmutablePair<>(getTaskName(), false);
+ long startTime = Time.monotonicNow();
+ try {
+ for (BucketLayout layout : Arrays.asList(BucketLayout.LEGACY,
+ BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+ // (HDDS-8580) Since "reprocess" iterate over the whole key table,
+ // containerKeyMap needs to be incrementally flushed to DB based on
+ // configured batch threshold.
+ // containerKeyCountMap can be flushed at the end since the number
+ // of containers in a cluster will not have significant memory
overhead.
+ Table<String, OmKeyInfo> omKeyInfoTable =
+ omMetadataManager.getKeyTable(layout);
+ try (
+ TableIterator<String, ? extends Table.KeyValue<String,
OmKeyInfo>>
+ keyIter = omKeyInfoTable.iterator()) {
+ while (keyIter.hasNext()) {
+ Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
+ OmKeyInfo omKeyInfo = kv.getValue();
+ handleKeyReprocess(kv.getKey(), omKeyInfo, containerKeyMap,
+ containerKeyCountMap);
+ if (!checkAndCallFlushToDB(containerKeyMap)) {
+ LOG.error("Unable to flush containerKey information to the
DB");
+ return new ImmutablePair<>(getTaskName(), false);
+ }
+ omKeyCount++;
}
- omKeyCount++;
}
}
+ } finally {
Review Comment:
For capturing task latency and excluding DB write latency, this is fine, but
DB flush latency should be captured on final flush and not every intermediate
flush.As of now it looks like db flush latency also being added in task
reprocess latency which is wrong. Do we want to capture with DB write latency ?
##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -87,45 +96,49 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
long omKeyCount = 0;
-
+ metrics.incrTaskReprocessCount();
Review Comment:
Shouldn't this metric be captured at the end of `reprocess` ?because
`reprocess` may fail, but we still incremented this metric ?
--
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]