This is an automated email from the ASF dual-hosted git repository.
erose pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new e280fbb6fed HDDS-13093. Add metrics for the cumulative state of
volumes (#8609)
e280fbb6fed is described below
commit e280fbb6fed92130a8a01f1cdbc5e9583216490d
Author: Rishabh Patel <[email protected]>
AuthorDate: Tue Aug 19 14:28:58 2025 -0700
HDDS-13093. Add metrics for the cumulative state of volumes (#8609)
---
.../container/common/volume/MutableVolumeSet.java | 28 +++++-
.../common/volume/VolumeHealthMetrics.java | 109 +++++++++++++++++++++
.../common/volume/TestPeriodicVolumeChecker.java | 9 ++
.../container/common/volume/TestVolumeSet.java | 29 +++++-
4 files changed, 171 insertions(+), 4 deletions(-)
diff --git
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
index b5f27b316a1..c704245be54 100644
---
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
+++
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
@@ -84,6 +84,7 @@ public class MutableVolumeSet implements VolumeSet {
private final StorageVolumeFactory volumeFactory;
private final StorageVolume.VolumeType volumeType;
private int maxVolumeFailuresTolerated;
+ private final VolumeHealthMetrics volumeHealthMetrics;
public MutableVolumeSet(String dnUuid, ConfigurationSource conf,
StateContext context, StorageVolume.VolumeType volumeType,
@@ -123,7 +124,14 @@ public MutableVolumeSet(String dnUuid, String clusterID,
maxVolumeFailuresTolerated = dnConf.getFailedDataVolumesTolerated();
}
- initializeVolumeSet();
+ // Ensure metrics are unregistered if the volume set initialization fails.
+ this.volumeHealthMetrics = VolumeHealthMetrics.create(volumeType);
+ try {
+ initializeVolumeSet();
+ } catch (Exception e) {
+ volumeHealthMetrics.unregister();
+ throw e;
+ }
}
public void setFailedVolumeListener(CheckedRunnable<IOException> runnable) {
@@ -174,7 +182,9 @@ private void initializeVolumeSet() throws IOException {
}
volumeMap.put(volume.getStorageDir().getPath(), volume);
volumeStateMap.get(volume.getStorageType()).add(volume);
+ volumeHealthMetrics.incrementHealthyVolumes();
} catch (IOException e) {
+ volumeHealthMetrics.incrementFailedVolumes();
if (volume != null) {
volume.shutdown();
}
@@ -337,6 +347,7 @@ private boolean addVolume(String volumeRoot, StorageType
storageType) {
} else {
if (failedVolumeMap.containsKey(volumeRoot)) {
failedVolumeMap.remove(volumeRoot);
+ volumeHealthMetrics.decrementFailedVolumes();
}
StorageVolume volume =
@@ -347,6 +358,7 @@ private boolean addVolume(String volumeRoot, StorageType
storageType) {
LOG.info("Added Volume : {} to VolumeSet",
volume.getStorageDir().getPath());
success = true;
+ volumeHealthMetrics.incrementHealthyVolumes();
}
} catch (IOException ex) {
LOG.error("Failed to add volume " + volumeRoot + " to VolumeSet", ex);
@@ -368,7 +380,8 @@ public void failVolume(String volumeRoot) {
volumeMap.remove(volumeRoot);
volumeStateMap.get(volume.getStorageType()).remove(volume);
failedVolumeMap.put(volumeRoot, volume);
-
+ volumeHealthMetrics.decrementHealthyVolumes();
+ volumeHealthMetrics.incrementFailedVolumes();
LOG.info("Moving Volume : {} to failed Volumes", volumeRoot);
} else if (failedVolumeMap.containsKey(volumeRoot)) {
LOG.info("Volume : {} is not active", volumeRoot);
@@ -390,10 +403,11 @@ public void removeVolume(String volumeRoot) throws
IOException {
volumeMap.remove(volumeRoot);
volumeStateMap.get(volume.getStorageType()).remove(volume);
-
+ volumeHealthMetrics.decrementHealthyVolumes();
LOG.info("Removed Volume : {} from VolumeSet", volumeRoot);
} else if (failedVolumeMap.containsKey(volumeRoot)) {
failedVolumeMap.remove(volumeRoot);
+ volumeHealthMetrics.decrementFailedVolumes();
LOG.info("Removed Volume : {} from failed VolumeSet", volumeRoot);
} else {
LOG.warn("Volume : {} does not exist in VolumeSet", volumeRoot);
@@ -415,6 +429,10 @@ public void shutdown() {
}
}
volumeMap.clear();
+
+ if (volumeHealthMetrics != null) {
+ volumeHealthMetrics.unregister();
+ }
}
@Override
@@ -483,4 +501,8 @@ public StorageVolume.VolumeType getVolumeType() {
return volumeType;
}
+ @VisibleForTesting
+ public VolumeHealthMetrics getVolumeHealthMetrics() {
+ return volumeHealthMetrics;
+ }
}
diff --git
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeHealthMetrics.java
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeHealthMetrics.java
new file mode 100644
index 00000000000..54895b8ec00
--- /dev/null
+++
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeHealthMetrics.java
@@ -0,0 +1,109 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.container.common.volume;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * This class is used to track Volume Health metrics for all volumes on a
datanode.
+ */
+@Metrics(about = "Ozone Volume Health Metrics",
+ context = OzoneConsts.OZONE)
+public final class VolumeHealthMetrics implements MetricsSource {
+
+ private static final String SOURCE_BASENAME =
+ VolumeHealthMetrics.class.getSimpleName();
+
+ private static final MetricsInfo TOTAL_VOLUMES =
+ Interns.info("TotalVolumes", "Total number of volumes");
+ private static final MetricsInfo HEALTHY_VOLUMES =
+ Interns.info("NumHealthyVolumes", "Number of healthy volumes");
+ private static final MetricsInfo FAILED_VOLUMES =
+ Interns.info("NumFailedVolumes", "Number of failed volumes");
+
+ private final MetricsRegistry registry;
+ private final String metricsSourceName;
+ private final AtomicInteger healthyVolumes;
+ private final AtomicInteger failedVolumes;
+
+ /**
+ * Constructor for VolumeHealthMetrics.
+ *
+ * @param volumeType Type of volumes (DATA_VOLUME, META_VOLUME, DB_VOLUME)
+ */
+ private VolumeHealthMetrics(StorageVolume.VolumeType volumeType) {
+ this.healthyVolumes = new AtomicInteger(0);
+ this.failedVolumes = new AtomicInteger(0);
+ metricsSourceName = SOURCE_BASENAME + '-' + volumeType.name();
+ registry = new MetricsRegistry(metricsSourceName);
+ }
+
+ /**
+ * Creates and registers a new VolumeHealthMetrics instance.
+ *
+ * @param volumeType Type of volumes (DATA_VOLUME, META_VOLUME, DB_VOLUME)
+ * @return The registered VolumeHealthMetrics instance
+ */
+ public static VolumeHealthMetrics create(StorageVolume.VolumeType
volumeType) {
+ MetricsSystem ms = DefaultMetricsSystem.instance();
+ VolumeHealthMetrics metrics = new VolumeHealthMetrics(volumeType);
+ return ms.register(metrics.metricsSourceName, "Volume Health Statistics",
metrics);
+ }
+
+ public void unregister() {
+ MetricsSystem ms = DefaultMetricsSystem.instance();
+ ms.unregisterSource(metricsSourceName);
+ }
+
+ public void incrementHealthyVolumes() {
+ healthyVolumes.incrementAndGet();
+ }
+
+ public void incrementFailedVolumes() {
+ failedVolumes.incrementAndGet();
+ }
+
+ public void decrementHealthyVolumes() {
+ healthyVolumes.decrementAndGet();
+ }
+
+ public void decrementFailedVolumes() {
+ failedVolumes.decrementAndGet();
+ }
+
+ @Override
+ public void getMetrics(MetricsCollector collector, boolean all) {
+ MetricsRecordBuilder builder = collector.addRecord(metricsSourceName);
+ registry.snapshot(builder, all);
+
+ builder
+ .addGauge(TOTAL_VOLUMES, healthyVolumes.get() + failedVolumes.get())
+ .addGauge(HEALTHY_VOLUMES, healthyVolumes.get())
+ .addGauge(FAILED_VOLUMES, failedVolumes.get());
+ }
+}
diff --git
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestPeriodicVolumeChecker.java
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestPeriodicVolumeChecker.java
index 2cdd06fa51b..e99d6461aae 100644
---
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestPeriodicVolumeChecker.java
+++
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestPeriodicVolumeChecker.java
@@ -19,6 +19,7 @@
import static
org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult.HEALTHY;
import static
org.apache.hadoop.ozone.container.common.volume.TestStorageVolumeChecker.makeVolumes;
+import static
org.apache.hadoop.ozone.container.common.volume.TestVolumeSet.assertNumVolumes;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.IOException;
@@ -91,6 +92,8 @@ public void testPeriodicVolumeChecker(TestInfo testInfo)
throws Exception {
assertEquals(0, metrics.getNumDataVolumeScans());
assertEquals(0, metrics.getNumMetadataVolumeScans());
assertEquals(0, metrics.getNumVolumesScannedInLastIteration());
+ assertNumVolumes(volumeSet, 1, 0);
+ assertNumVolumes(metaVolumeSet, 1, 0);
// first round
timer.advance(gap.toMillis() / 3);
@@ -101,6 +104,8 @@ public void testPeriodicVolumeChecker(TestInfo testInfo)
throws Exception {
assertEquals(1, metrics.getNumMetadataVolumeScans());
assertEquals(5, metrics.getNumVolumesScannedInLastIteration());
assertEquals(0, metrics.getNumIterationsSkipped());
+ assertNumVolumes(volumeSet, 1, 0);
+ assertNumVolumes(metaVolumeSet, 1, 0);
// periodic disk checker next round within gap
timer.advance(gap.toMillis() / 3);
@@ -112,6 +117,8 @@ public void testPeriodicVolumeChecker(TestInfo testInfo)
throws Exception {
assertEquals(1, metrics.getNumMetadataVolumeScans());
assertEquals(5, metrics.getNumVolumesScannedInLastIteration());
assertEquals(1, metrics.getNumIterationsSkipped());
+ assertNumVolumes(volumeSet, 1, 0);
+ assertNumVolumes(metaVolumeSet, 1, 0);
// periodic disk checker next round
timer.advance(interval.toMillis());
@@ -122,6 +129,8 @@ public void testPeriodicVolumeChecker(TestInfo testInfo)
throws Exception {
assertEquals(2, metrics.getNumMetadataVolumeScans());
assertEquals(5, metrics.getNumVolumesScannedInLastIteration());
assertEquals(1, metrics.getNumIterationsSkipped());
+ assertNumVolumes(volumeSet, 1, 0);
+ assertNumVolumes(metaVolumeSet, 1, 0);
} finally {
volumeChecker.shutdownAndWait(1, TimeUnit.SECONDS);
}
diff --git
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
index 51b4e68257f..19ccb492b96 100644
---
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
+++
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
@@ -19,6 +19,8 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
import static
org.apache.hadoop.ozone.container.common.volume.HddsVolume.HDDS_VOLUME_DIR;
+import static org.apache.ozone.test.MetricsAsserts.assertGauge;
+import static org.apache.ozone.test.MetricsAsserts.getMetrics;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assumptions.assumeThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -35,6 +37,7 @@
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
@@ -101,6 +104,13 @@ private boolean checkVolumeExistsInVolumeSet(String
volumeRoot) {
return false;
}
+ static void assertNumVolumes(MutableVolumeSet volumeSet, int
expectedHealthyVolumes, int expectedFailedVolumes) {
+ MetricsRecordBuilder metricsRecords =
getMetrics(volumeSet.getVolumeHealthMetrics());
+ assertGauge("TotalVolumes", expectedHealthyVolumes +
expectedFailedVolumes, metricsRecords);
+ assertGauge("NumHealthyVolumes", expectedHealthyVolumes, metricsRecords);
+ assertGauge("NumFailedVolumes", expectedFailedVolumes, metricsRecords);
+ }
+
@Test
public void testVolumeSetInitialization() throws Exception {
@@ -113,6 +123,8 @@ public void testVolumeSetInitialization() throws Exception {
"VolumeSet not initialized correctly");
assertTrue(checkVolumeExistsInVolumeSet(volume2),
"VolumeSet not initialized correctly");
+
+ assertNumVolumes(volumeSet, 2, 0);
}
@Test
@@ -120,6 +132,8 @@ public void testAddVolume() {
assertEquals(2, volumeSet.getVolumesList().size());
+ assertNumVolumes(volumeSet, 2, 0);
+
// Add a volume to VolumeSet
String volume3 = baseDir.resolve("disk3").toString();
boolean success = volumeSet.addVolume(volume3);
@@ -128,10 +142,13 @@ public void testAddVolume() {
assertEquals(3, volumeSet.getVolumesList().size());
assertTrue(checkVolumeExistsInVolumeSet(volume3),
"AddVolume did not add requested volume to VolumeSet");
+
+ assertNumVolumes(volumeSet, 3, 0);
}
@Test
public void testFailVolume() throws Exception {
+ assertNumVolumes(volumeSet, 2, 0);
//Fail a volume
volumeSet.failVolume(HddsVolumeUtil.getHddsRoot(volume1));
@@ -148,10 +165,13 @@ public void testFailVolume() throws Exception {
// Failed volume should not exist in VolumeMap
assertThat(volumeSet.getVolumeMap()).doesNotContainKey(volume1);
+
+ assertNumVolumes(volumeSet, 1, 1);
}
@Test
public void testRemoveVolume() throws Exception {
+ assertNumVolumes(volumeSet, 2, 0);
assertEquals(2, volumeSet.getVolumesList().size());
@@ -159,6 +179,8 @@ public void testRemoveVolume() throws Exception {
volumeSet.removeVolume(HddsVolumeUtil.getHddsRoot(volume1));
assertEquals(1, volumeSet.getVolumesList().size());
+ assertNumVolumes(volumeSet, 1, 0);
+
// Attempting to remove a volume which does not exist in VolumeSet should
// log a warning.
LogCapturer logs = LogCapturer.captureLogs(MutableVolumeSet.class);
@@ -167,10 +189,13 @@ public void testRemoveVolume() throws Exception {
String expectedLogMessage = "Volume : " +
HddsVolumeUtil.getHddsRoot(volume1) + " does not exist in VolumeSet";
assertThat(logs.getOutput()).contains(expectedLogMessage);
+
+ assertNumVolumes(volumeSet, 1, 0);
}
@Test
public void testVolumeInInconsistentState() throws Exception {
+ assertNumVolumes(volumeSet, 2, 0);
assertEquals(2, volumeSet.getVolumesList().size());
// Add a volume to VolumeSet
@@ -195,6 +220,7 @@ public void testVolumeInInconsistentState() throws
Exception {
assertFalse(checkVolumeExistsInVolumeSet(volume3), "AddVolume should fail"
+
" for an inconsistent volume");
+ assertNumVolumes(volumeSet, 2, 0);
// Delete volume3
File volume = new File(volume3);
FileUtils.deleteDirectory(volume);
@@ -202,6 +228,7 @@ public void testVolumeInInconsistentState() throws
Exception {
@Test
public void testShutdown() throws Exception {
+ assertNumVolumes(volumeSet, 2, 0);
List<StorageVolume> volumesList = volumeSet.getVolumesList();
volumeSet.shutdown();
@@ -227,7 +254,7 @@ void testFailVolumes(@TempDir File readOnlyVolumePath,
@TempDir File volumePath)
assertEquals(1, volSet.getFailedVolumesList().size());
assertEquals(readOnlyVolumePath, volSet.getFailedVolumesList().get(0)
.getStorageDir());
-
+ assertNumVolumes(volSet, 1, 1);
volSet.shutdown();
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]