You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@celeborn.apache.org by GitBox <gi...@apache.org> on 2023/01/05 03:15:56 UTC

[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1127: [CELEBORN-168][FEATURE] Add disk usage related metrics for Worker

AngersZhuuuu commented on code in PR #1127:
URL: https://github.com/apache/incubator-celeborn/pull/1127#discussion_r1060262279


##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala:
##########
@@ -73,6 +74,24 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs
     }
   }
 
+  disksSnapshot()
+    .groupBy(_.deviceInfo)
+    .foreach { case (deviceInfo: DeviceInfo, diskInfos: List[DiskInfo]) =>
+      val dumbArray = Array("none", "0", "0", "0", "0%", "none")

Review Comment:
    you mean default value  free is 0%?



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala:
##########
@@ -73,6 +74,24 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs
     }
   }
 
+  disksSnapshot()
+    .groupBy(_.deviceInfo)
+    .foreach { case (deviceInfo: DeviceInfo, diskInfos: List[DiskInfo]) =>
+      val dumbArray = Array("none", "0", "0", "0", "0%", "none")
+      def usage: Array[String] =
+        Utils.tryWithTimeoutAndCallback(DeviceMonitor.getDiskUsageInfos(diskInfos.head))(dumbArray)(
+          deviceCheckThreadPool,
+          conf.workerDeviceStatusCheckTimeout,
+          s"Disk: ${diskInfos.head.mountPoint} Usage Check Timeout")

Review Comment:
   Also can we just use the value of device check? cc @waitinfuture 



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala:
##########
@@ -73,6 +74,24 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs
     }
   }
 
+  disksSnapshot()
+    .groupBy(_.deviceInfo)
+    .foreach { case (deviceInfo: DeviceInfo, diskInfos: List[DiskInfo]) =>
+      val dumbArray = Array("none", "0", "0", "0", "0%", "none")
+      def usage: Array[String] =
+        Utils.tryWithTimeoutAndCallback(DeviceMonitor.getDiskUsageInfos(diskInfos.head))(dumbArray)(
+          deviceCheckThreadPool,
+          conf.workerDeviceStatusCheckTimeout,
+          s"Disk: ${diskInfos.head.mountPoint} Usage Check Timeout")
+
+      workerSource.addGauge(
+        s"${WorkerSource.DeviceOSTotalCapacity}_${deviceInfo.name}",
+        _ => usage(usage.length - 5))
+      workerSource.addGauge(
+        s"${WorkerSource.DeviceOSFreeCapacity}_${deviceInfo.name}",
+        _ => usage(usage.length - 3))

Review Comment:
   Your code will cause the command to be run twice every time you get metrics.



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/WorkerSource.scala:
##########
@@ -91,4 +91,10 @@ object WorkerSource {
   val DiskBuffer = "DiskBuffer"
   val PausePushDataCount = "PausePushData"
   val PausePushDataAndReplicateCount = "PausePushDataAndReplicate"
+
+  // local device
+  val DeviceOSFreeCapacity = "DeviceOSFreeCapacity"
+  val DeviceOSTotalCapacity = "DeviceOSTotalCapacity"
+  val DeviceCelebornFreeCapacity = "DeviceCelebornFreeCapacity"
+  val DeviceCelebornTotalCapacity = "DeviceCelebornTotalCapacity"

Review Comment:
   We should add a size unit in the name such as  `DeviceOSFreeCapacity(GB)` to indicate the size



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/DeviceMonitor.scala:
##########
@@ -203,6 +224,11 @@ object DeviceMonitor {
     }
   }
 
+  def getDiskUsageInfos(diskInfo: DiskInfo): Array[String] = {
+    // TODO: will it be more flexible if return as Bytes?

Review Comment:
   +1



-- 
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: issues-unsubscribe@celeborn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org