This is an automated email from the ASF dual-hosted git repository.

ethanfeng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/celeborn.git


The following commit(s) were added to refs/heads/main by this push:
     new 59163c2a2 [CELEBORN-1745] Remove application top disk usage code
59163c2a2 is described below

commit 59163c2a233be8e58a184f1b0b33099c69853984
Author: Wang, Fei <[email protected]>
AuthorDate: Thu Nov 28 10:55:34 2024 +0800

    [CELEBORN-1745] Remove application top disk usage code
    
    ### What changes were proposed in this pull request?
    Remove the code for app top disk usage both in master and worker end.
    
    Prefer to use below prometheus expr to figure out the top app usages.
    ```
    topk(50, sum by (applicationId) 
(metrics_diskBytesWritten_Value{role="worker", applicationId!=""}))
    ```
    
    ### Why are the changes needed?
    To address comments: 
https://github.com/apache/celeborn/pull/2947#issuecomment-2499564978
    
    > Due to the application dimension resource consumption, this feature 
should be included in the deprecated features. Maybe you can remove the codes 
for application top disk usage.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, remove the app top disk usage api.
    
    ### How was this patch tested?
    GA.
    
    Closes #2949 from turboFei/remove_app_top_usage.
    
    Authored-by: Wang, Fei <[email protected]>
    Signed-off-by: mingji <[email protected]>
---
 .../apache/celeborn/cli/master/MasterOptions.scala |   5 -
 .../celeborn/cli/master/MasterSubcommand.scala     |   2 -
 .../celeborn/cli/master/MasterSubcommandImpl.scala |   4 -
 .../apache/celeborn/cli/worker/WorkerOptions.scala |   5 -
 .../celeborn/cli/worker/WorkerSubcommand.scala     |   2 -
 .../celeborn/cli/worker/WorkerSubcommandImpl.scala |   4 -
 .../celeborn/cli/TestCelebornCliCommands.scala     |  10 --
 common/src/main/proto/TransportMessages.proto      |   6 +-
 .../org/apache/celeborn/common/CelebornConf.scala  |  27 ---
 .../celeborn/common/meta/AppDiskUsageMetric.scala  | 189 ---------------------
 .../common/protocol/message/ControlMessages.scala  |   8 -
 .../apache/celeborn/common/util/PbSerDeUtils.scala |  52 +-----
 docs/configuration/metrics.md                      |   3 -
 docs/developers/sbt.md                             |   6 +-
 docs/migration.md                                  |   7 +-
 docs/restapi.md                                    |   2 -
 .../master/clustermeta/AbstractMetaManager.java    |  16 --
 .../master/clustermeta/IMetadataHandler.java       |   1 -
 .../clustermeta/SingleMasterMetaManager.java       |   4 -
 .../master/clustermeta/ha/HAMasterMetaManager.java |   4 -
 .../deploy/master/clustermeta/ha/MetaHandler.java  |   4 -
 master/src/main/proto/Resource.proto               |   2 +-
 .../celeborn/service/deploy/master/Master.scala    |  11 --
 .../master/http/api/v1/ApplicationResource.scala   |  31 +---
 .../clustermeta/DefaultMetaSystemSuiteJ.java       |   4 -
 .../clustermeta/ha/MasterStateMachineSuiteJ.java   |  26 ---
 .../ha/RatisMasterStatusSystemSuiteJ.java          |   4 -
 .../deploy/master/AppDiskUsageMetricSuite.scala    | 124 --------------
 .../http/api/v1/ApiV1MasterResourceSuite.scala     |   7 +-
 .../celeborn/rest/v1/master/ApplicationApi.java    |  68 --------
 .../celeborn/rest/v1/model/AppDiskUsageData.java   | 170 ------------------
 .../rest/v1/model/AppDiskUsageSnapshotData.java    | 182 --------------------
 .../v1/model/AppDiskUsageSnapshotsResponse.java    | 120 -------------
 .../rest/v1/model/AppDiskUsagesResponse.java       | 120 -------------
 .../celeborn/rest/v1/worker/ApplicationApi.java    |  68 --------
 .../src/main/openapi3/master_rest_v1.yaml          |  61 -------
 .../src/main/openapi3/worker_rest_v1.yaml          |  43 -----
 .../celeborn/server/common/HttpService.scala       |   2 -
 .../server/common/http/api/ApiBaseResource.scala   |  13 --
 .../server/common/http/ApiBaseResourceSuite.scala  |   5 -
 .../celeborn/service/deploy/worker/Worker.scala    |  17 +-
 .../worker/http/api/v1/ApplicationResource.scala   |  27 +--
 .../deploy/worker/storage/StorageManager.scala     |  15 --
 .../http/api/v1/ApiV1OpenapiClientSuite.scala      |   1 -
 .../api/v1/ApiV1WorkerOpenapiClientSuite.scala     |   1 -
 .../http/api/v1/ApiV1WorkerResourceSuite.scala     |   7 +-
 46 files changed, 20 insertions(+), 1470 deletions(-)

diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterOptions.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterOptions.scala
index 840a54d23..2dbff0ebb 100644
--- a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterOptions.scala
+++ b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterOptions.scala
@@ -30,11 +30,6 @@ final class MasterOptions {
   @Option(names = Array("--show-cluster-shuffles"), description = Array("Show 
cluster shuffles"))
   private[master] var showClusterShuffles: Boolean = _
 
-  @Option(
-    names = Array("--show-top-disk-used-apps"),
-    description = Array("Show top disk used apps"))
-  private[master] var showTopDiskUsedApps: Boolean = _
-
   @Option(names = Array("--exclude-worker"), description = Array("Exclude 
workers by ID"))
   private[master] var excludeWorkers: Boolean = _
 
diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommand.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommand.scala
index a875621e2..d976f0c6b 100644
--- a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommand.scala
+++ b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommand.scala
@@ -77,8 +77,6 @@ trait MasterSubcommand extends CliLogging {
 
   private[master] def runShowClusterShuffles: ShufflesResponse
 
-  private[master] def runShowTopDiskUsedApps: AppDiskUsageSnapshotsResponse
-
   private[master] def runExcludeWorkers: HandleResponse
 
   private[master] def runRemoveExcludedWorkers: HandleResponse
diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommandImpl.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommandImpl.scala
index 7dbc67f3f..da1479a10 100644
--- 
a/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommandImpl.scala
+++ 
b/cli/src/main/scala/org/apache/celeborn/cli/master/MasterSubcommandImpl.scala
@@ -34,7 +34,6 @@ class MasterSubcommandImpl extends Runnable with 
MasterSubcommand {
     if (masterOptions.showMastersInfo) log(runShowMastersInfo)
     if (masterOptions.showClusterApps) log(runShowClusterApps)
     if (masterOptions.showClusterShuffles) log(runShowClusterShuffles)
-    if (masterOptions.showTopDiskUsedApps) log(runShowTopDiskUsedApps)
     if (masterOptions.excludeWorkers) log(runExcludeWorkers)
     if (masterOptions.removeExcludedWorkers) log(runRemoveExcludedWorkers)
     if (masterOptions.removeWorkersUnavailableInfo) 
log(runRemoveWorkersUnavailableInfo)
@@ -67,9 +66,6 @@ class MasterSubcommandImpl extends Runnable with 
MasterSubcommand {
 
   private[master] def runShowClusterShuffles: ShufflesResponse = 
shuffleApi.getShuffles
 
-  private[master] def runShowTopDiskUsedApps: AppDiskUsageSnapshotsResponse =
-    applicationApi.getApplicationsDiskUsageSnapshots
-
   private[master] def runExcludeWorkers: HandleResponse = {
     val workerIds = getWorkerIds
     val excludeWorkerRequest = new ExcludeWorkerRequest().add(workerIds)
diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerOptions.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerOptions.scala
index baeb695ab..5b776e835 100644
--- a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerOptions.scala
+++ b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerOptions.scala
@@ -34,11 +34,6 @@ final class WorkerOptions {
     description = Array("Show shuffles running on the worker"))
   private[worker] var showShufflesOnWorker: Boolean = _
 
-  @Option(
-    names = Array("--show-top-disk-used-apps"),
-    description = Array("Show top disk used applications"))
-  private[worker] var showTopDiskUsedApps: Boolean = _
-
   @Option(
     names = Array("--show-partition-location-info"),
     description = Array("Show partition location information"))
diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommand.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommand.scala
index bc70f8e01..204a9c896 100644
--- a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommand.scala
+++ b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommand.scala
@@ -62,8 +62,6 @@ trait WorkerSubcommand extends CliLogging {
 
   private[worker] def runShowShufflesOnWorker: ShufflesResponse
 
-  private[worker] def runShowTopDiskUsedApps: AppDiskUsagesResponse
-
   private[worker] def runShowPartitionLocationInfo: ShufflePartitionsResponse
 
   private[worker] def runShowUnavailablePeers: UnAvailablePeersResponse
diff --git 
a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommandImpl.scala 
b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommandImpl.scala
index 214e009a4..dd542bdee 100644
--- 
a/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommandImpl.scala
+++ 
b/cli/src/main/scala/org/apache/celeborn/cli/worker/WorkerSubcommandImpl.scala
@@ -29,7 +29,6 @@ class WorkerSubcommandImpl extends Runnable with 
WorkerSubcommand {
     if (workerOptions.showWorkerInfo) log(runShowWorkerInfo)
     if (workerOptions.showAppsOnWorker) log(runShowAppsOnWorker)
     if (workerOptions.showShufflesOnWorker) log(runShowShufflesOnWorker)
-    if (workerOptions.showTopDiskUsedApps) log(runShowTopDiskUsedApps)
     if (workerOptions.showPartitionLocationInfo) 
log(runShowPartitionLocationInfo)
     if (workerOptions.showUnavailablePeers) log(runShowUnavailablePeers)
     if (workerOptions.isShutdown) log(runIsShutdown)
@@ -48,9 +47,6 @@ class WorkerSubcommandImpl extends Runnable with 
WorkerSubcommand {
 
   private[worker] def runShowShufflesOnWorker: ShufflesResponse = 
shuffleApi.getShuffles
 
-  private[worker] def runShowTopDiskUsedApps: AppDiskUsagesResponse =
-    applicationApi.getApplicationsDiskUsage
-
   private[worker] def runShowPartitionLocationInfo: ShufflePartitionsResponse =
     shuffleApi.getShufflePartitions
 
diff --git 
a/cli/src/test/scala/org/apache/celeborn/cli/TestCelebornCliCommands.scala 
b/cli/src/test/scala/org/apache/celeborn/cli/TestCelebornCliCommands.scala
index 1f450a19f..8a8f881c2 100644
--- a/cli/src/test/scala/org/apache/celeborn/cli/TestCelebornCliCommands.scala
+++ b/cli/src/test/scala/org/apache/celeborn/cli/TestCelebornCliCommands.scala
@@ -84,11 +84,6 @@ class TestCelebornCliCommands extends CelebornFunSuite with 
MiniClusterFeature {
     captureOutputAndValidateResponse(args, "ShufflesResponse")
   }
 
-  test("worker --show-top-disk-used-apps") {
-    val args = prepareWorkerArgs() :+ "--show-top-disk-used-apps"
-    captureOutputAndValidateResponse(args, "AppDiskUsagesResponse")
-  }
-
   test("worker --show-partition-location-info") {
     val args = prepareWorkerArgs() :+ "--show-partition-location-info"
     captureOutputAndValidateResponse(args, "ShufflePartitionsResponse")
@@ -151,11 +146,6 @@ class TestCelebornCliCommands extends CelebornFunSuite 
with MiniClusterFeature {
     captureOutputAndValidateResponse(args, "ShufflesResponse")
   }
 
-  test("master --show-top-disk-used-apps") {
-    val args = prepareMasterArgs() :+ "--show-top-disk-used-apps"
-    captureOutputAndValidateResponse(args, "AppDiskUsageSnapshotsResponse")
-  }
-
   test("master --show-worker-event-info") {
     val args = prepareMasterArgs() :+ "--show-worker-event-info"
     captureOutputAndValidateResponse(args, "WorkerEventsResponse")
diff --git a/common/src/main/proto/TransportMessages.proto 
b/common/src/main/proto/TransportMessages.proto
index be09e83c7..5c03ba291 100644
--- a/common/src/main/proto/TransportMessages.proto
+++ b/common/src/main/proto/TransportMessages.proto
@@ -646,11 +646,13 @@ message PbResourceConsumption {
   map<string, PbResourceConsumption> subResourceConsumptions = 5;
 }
 
+// deprecated
 message PbAppDiskUsage {
   string appId = 1;
   int64 estimatedUsage = 2;
 }
 
+// deprecated
 message PbAppDiskUsageSnapshot {
   int32 topItemCount = 1;
   repeated PbAppDiskUsage topNItems = 2;
@@ -668,8 +670,8 @@ message PbSnapshotMetaInfo {
   repeated PbWorkerInfo workers = 7;
   int64 partitionTotalWritten = 8;
   int64 partitionTotalFileCount = 9;
-  repeated PbAppDiskUsageSnapshot appDiskUsageMetricSnapshots = 10;
-  PbAppDiskUsageSnapshot currentAppDiskUsageMetricsSnapshot = 11;
+  repeated PbAppDiskUsageSnapshot appDiskUsageMetricSnapshots = 10; // 
deprecated
+  PbAppDiskUsageSnapshot currentAppDiskUsageMetricsSnapshot = 11; // deprecated
   map<string, int64> lostWorkers = 12;
   repeated PbWorkerInfo shutdownWorkers = 13;
   repeated PbWorkerInfo manuallyExcludedWorkers = 14;
diff --git 
a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala 
b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala
index 976736f5d..c8274ae80 100644
--- a/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala
+++ b/common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala
@@ -874,9 +874,6 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable 
with Logging with Se
   def metricsCapacity: Int = get(METRICS_CAPACITY)
   def metricsExtraLabels: Map[String, String] =
     get(METRICS_EXTRA_LABELS).map(Utils.parseMetricLabels).toMap
-  def metricsAppTopDiskUsageCount: Int = get(METRICS_APP_TOP_DISK_USAGE_COUNT)
-  def metricsAppTopDiskUsageWindowSize: Int = 
get(METRICS_APP_TOP_DISK_USAGE_WINDOW_SIZE)
-  def metricsAppTopDiskUsageInterval: Long = 
get(METRICS_APP_TOP_DISK_USAGE_INTERVAL)
   def metricsWorkerAppTopResourceConsumptionCount: Int =
     get(METRICS_WORKER_APP_TOP_RESOURCE_CONSUMPTION_COUNT)
   def metricsWorkerForceAppendPauseSpentTimeThreshold: Int =
@@ -5282,30 +5279,6 @@ object CelebornConf extends Logging {
         "Allowed pattern is: 
`<label1_key>:<label1_value>[,<label2_key>:<label2_value>]*`")
       .createWithDefault(Seq.empty)
 
-  val METRICS_APP_TOP_DISK_USAGE_COUNT: ConfigEntry[Int] =
-    buildConf("celeborn.metrics.app.topDiskUsage.count")
-      .categories("metrics")
-      .doc("Size for top items about top disk usage applications list.")
-      .version("0.2.0")
-      .intConf
-      .createWithDefault(50)
-
-  val METRICS_APP_TOP_DISK_USAGE_WINDOW_SIZE: ConfigEntry[Int] =
-    buildConf("celeborn.metrics.app.topDiskUsage.windowSize")
-      .categories("metrics")
-      .doc("Window size about top disk usage application list.")
-      .version("0.2.0")
-      .intConf
-      .createWithDefault(24)
-
-  val METRICS_APP_TOP_DISK_USAGE_INTERVAL: ConfigEntry[Long] =
-    buildConf("celeborn.metrics.app.topDiskUsage.interval")
-      .categories("metrics")
-      .doc("Time length for a window about top disk usage application list.")
-      .version("0.2.0")
-      .timeConf(TimeUnit.SECONDS)
-      .createWithDefaultString("10min")
-
   val METRICS_WORKER_APP_TOP_RESOURCE_CONSUMPTION_COUNT: ConfigEntry[Int] =
     buildConf("celeborn.metrics.worker.app.topResourceConsumption.count")
       .categories("metrics")
diff --git 
a/common/src/main/scala/org/apache/celeborn/common/meta/AppDiskUsageMetric.scala
 
b/common/src/main/scala/org/apache/celeborn/common/meta/AppDiskUsageMetric.scala
deleted file mode 100644
index cd6bc1066..000000000
--- 
a/common/src/main/scala/org/apache/celeborn/common/meta/AppDiskUsageMetric.scala
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.celeborn.common.meta
-
-import java.time.{Instant, LocalDateTime, ZoneId}
-import java.util.Objects
-import java.util.concurrent.TimeUnit
-import java.util.concurrent.atomic.AtomicReference
-
-import scala.collection.JavaConverters.mapAsScalaMapConverter
-
-import org.apache.celeborn.common.CelebornConf
-import org.apache.celeborn.common.internal.Logging
-import org.apache.celeborn.common.util.{ThreadUtils, Utils}
-
-case class AppDiskUsage(var appId: String, var estimatedUsage: Long) {
-  override def toString: String =
-    s"Application $appId used approximate 
${Utils.bytesToString(estimatedUsage)}"
-}
-
-class AppDiskUsageSnapShot(val topItemCount: Int) extends Logging with 
Serializable {
-  var topNItems = new Array[AppDiskUsage](topItemCount)
-  var startSnapShotTime = System.currentTimeMillis()
-  var endSnapShotTime: Long = _
-
-  def commit(): Unit = {
-    endSnapShotTime = System.currentTimeMillis()
-  }
-
-  def updateAppDiskUsage(appId: String, usage: Long): Unit = {
-    val dropIndex = topNItems.indexWhere(usage => usage != null && usage.appId 
== appId)
-    if (dropIndex != -1) {
-      drop(dropIndex)
-    }
-    val insertIndex = findInsertPosition(usage)
-    if (insertIndex != -1) {
-      shift(insertIndex)
-      topNItems(insertIndex) = AppDiskUsage(appId, usage)
-    }
-  }
-
-  def shift(index: Int): Unit = {
-    for (i <- topItemCount - 1 until index by -1) {
-      topNItems(i) = topNItems(i - 1)
-    }
-  }
-
-  def drop(index: Int): Unit = {
-    for (i <- index until topItemCount - 1) {
-      topNItems(i) = topNItems(i + 1)
-    }
-    topNItems(topItemCount - 1) = null
-  }
-
-  def findInsertPosition(usage: Long): Int = {
-    if (topNItems(0) == null) {
-      return 0
-    }
-    for (i <- 0 until topItemCount) {
-      if (topNItems(i) == null || topNItems(i).estimatedUsage < usage) {
-        return i
-      }
-    }
-    -1
-  }
-
-  def restoreFromSnapshot(array: Array[AppDiskUsage]): Unit = {
-    // Restored snapshots only contains values not null
-    for (i <- 0 until topItemCount) {
-      if (i < array.length) {
-        topNItems(i) = array(i)
-      } else {
-        topNItems(i) = null
-      }
-    }
-  }
-
-  override def equals(obj: Any): Boolean = {
-    obj.isInstanceOf[AppDiskUsageSnapShot] &&
-    obj.asInstanceOf[AppDiskUsageSnapShot].topItemCount == topItemCount &&
-    obj.asInstanceOf[AppDiskUsageSnapShot].startSnapShotTime == 
startSnapShotTime &&
-    obj.asInstanceOf[AppDiskUsageSnapShot].endSnapShotTime == endSnapShotTime 
&&
-    obj.asInstanceOf[AppDiskUsageSnapShot].topNItems.zip(topNItems).forall { 
case (x, y) => x == y }
-  }
-
-  override def hashCode(): Int = {
-    Objects.hashCode(topItemCount, startSnapShotTime, endSnapShotTime, 
topNItems)
-  }
-
-  override def toString(): String = {
-    val zoneId = ZoneId.systemDefault()
-    s"Snapshot " +
-      s"start 
${LocalDateTime.ofInstant(Instant.ofEpochMilli(startSnapShotTime), zoneId)} " +
-      s"end ${LocalDateTime.ofInstant(Instant.ofEpochMilli(endSnapShotTime), 
zoneId)}" +
-      s" ${topNItems.filter(_ != null).mkString(", ")}"
-  }
-}
-
-// This metric collects approximate value because worker won't report all app 
disk usage value for reducing memory pressure. .
-class AppDiskUsageMetric(conf: CelebornConf) extends Logging {
-  val usageCount = conf.metricsAppTopDiskUsageCount
-  val snapshotCount = conf.metricsAppTopDiskUsageWindowSize
-  val interval = conf.metricsAppTopDiskUsageInterval
-  val snapShots = new Array[AppDiskUsageSnapShot](snapshotCount)
-  val logExecutor =
-    
ThreadUtils.newDaemonSingleThreadScheduledExecutor("master-app-disk-usage-metrics-logger")
-  val updateExecutor =
-    
ThreadUtils.newDaemonSingleThreadExecutor("master-app-disk-usage-metrics-updater")
-  var currentSnapShot: AtomicReference[AppDiskUsageSnapShot] =
-    new AtomicReference[AppDiskUsageSnapShot]()
-
-  def update(appDiskUsage: java.util.Map[String, java.lang.Long]): Unit = {
-    updateExecutor.submit(new Runnable {
-      override def run(): Unit = {
-        if (currentSnapShot.get() != null) {
-          appDiskUsage.asScala.foreach { case (key, usage) =>
-            currentSnapShot.get().updateAppDiskUsage(key, usage)
-          }
-        }
-      }
-    })
-  }
-
-  logExecutor.scheduleWithFixedDelay(
-    new Runnable {
-      override def run(): Unit = {
-        if (currentSnapShot.get() != null) {
-          currentSnapShot.get().commit()
-        }
-        currentSnapShot.set(getNewSnapShot())
-        val summaryStr = Some(summary()).getOrElse("")
-        logInfo(s"App Disk Usage Top$usageCount Report: $summaryStr")
-      }
-    },
-    60,
-    interval,
-    TimeUnit.SECONDS)
-
-  def getNewSnapShot(): AppDiskUsageSnapShot = {
-    for (i <- snapshotCount - 1 until 0 by -1) {
-      snapShots(i) = snapShots(i - 1)
-    }
-    snapShots(0) = new AppDiskUsageSnapShot(usageCount)
-    snapShots(0)
-  }
-
-  def summary(): String = {
-    val stringBuilder = new StringBuilder()
-    for (i <- 0 until snapshotCount) {
-      if (snapShots(i) != null && snapShots(i).topNItems.exists(_ != null)) {
-        stringBuilder.append("\n")
-        stringBuilder.append(snapShots(i))
-      }
-    }
-    stringBuilder.toString()
-  }
-
-  def restoreFromSnapshot(array: Array[AppDiskUsageSnapShot]): Unit = {
-    // Restored snapshots only contains values not null
-    for (i <- 0 until snapshotCount) {
-      if (i < array.length) {
-        snapShots(i) = array(i)
-      } else {
-        snapShots(i) = null
-      }
-    }
-  }
-
-  def topSnapshots(): Seq[AppDiskUsageSnapShot] = {
-    snapShots.take(snapshotCount)
-      .filter(_ != null)
-      .filter(_.topNItems.exists(_ != null))
-  }
-}
diff --git 
a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala
 
b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala
index 3ec9c699b..7b2c3b132 100644
--- 
a/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala
+++ 
b/common/src/main/scala/org/apache/celeborn/common/protocol/message/ControlMessages.scala
@@ -118,7 +118,6 @@ object ControlMessages extends Logging {
       disks: Seq[DiskInfo],
       userResourceConsumption: util.Map[UserIdentifier, ResourceConsumption],
       activeShuffleKeys: util.Set[String],
-      estimatedAppDiskUsage: util.HashMap[String, java.lang.Long],
       highWorkload: Boolean,
       workerStatus: WorkerStatus,
       override var requestId: String = ZERO_UUID) extends MasterRequestMessage
@@ -612,7 +611,6 @@ object ControlMessages extends Logging {
           disks,
           userResourceConsumption,
           activeShuffleKeys,
-          estimatedAppDiskUsage,
           highWorkload,
           workerStatus,
           requestId) =>
@@ -628,7 +626,6 @@ object ControlMessages extends Logging {
         .putAllUserResourceConsumption(pbUserResourceConsumption)
         .setReplicatePort(replicatePort)
         .addAllActiveShuffleKeys(activeShuffleKeys)
-        .putAllEstimatedAppDiskUsage(estimatedAppDiskUsage)
         .setHighWorkload(highWorkload)
         .setWorkerStatus(PbSerDeUtils.toPbWorkerStatus(workerStatus))
         .setRequestId(requestId)
@@ -1048,14 +1045,10 @@ object ControlMessages extends Logging {
 
       case HEARTBEAT_FROM_WORKER_VALUE =>
         val pbHeartbeatFromWorker = 
PbHeartbeatFromWorker.parseFrom(message.getPayload)
-        val estimatedAppDiskUsage = new util.HashMap[String, java.lang.Long]()
         val userResourceConsumption = 
PbSerDeUtils.fromPbUserResourceConsumption(
           pbHeartbeatFromWorker.getUserResourceConsumptionMap)
         val pbDisks =
           
pbHeartbeatFromWorker.getDisksList.asScala.toSeq.map(PbSerDeUtils.fromPbDiskInfo)
-        if (!pbHeartbeatFromWorker.getEstimatedAppDiskUsageMap.isEmpty) {
-          
estimatedAppDiskUsage.putAll(pbHeartbeatFromWorker.getEstimatedAppDiskUsageMap)
-        }
         val activeShuffleKeys = new util.HashSet[String]()
         if (!pbHeartbeatFromWorker.getActiveShuffleKeysList.isEmpty) {
           
activeShuffleKeys.addAll(pbHeartbeatFromWorker.getActiveShuffleKeysList)
@@ -1072,7 +1065,6 @@ object ControlMessages extends Logging {
           pbDisks,
           userResourceConsumption,
           activeShuffleKeys,
-          estimatedAppDiskUsage,
           pbHeartbeatFromWorker.getHighWorkload,
           workerStatus,
           pbHeartbeatFromWorker.getRequestId)
diff --git 
a/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala 
b/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala
index 0c638f241..d5f2019e5 100644
--- a/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala
+++ b/common/src/main/scala/org/apache/celeborn/common/util/PbSerDeUtils.scala
@@ -19,14 +19,13 @@ package org.apache.celeborn.common.util
 
 import java.util
 import java.util.concurrent.ConcurrentHashMap
-import java.util.function.IntFunction
 
 import scala.collection.JavaConverters._
 
 import com.google.protobuf.InvalidProtocolBufferException
 
 import org.apache.celeborn.common.identity.UserIdentifier
-import org.apache.celeborn.common.meta.{AppDiskUsage, AppDiskUsageSnapShot, 
ApplicationMeta, DeviceInfo, DiskFileInfo, DiskInfo, MapFileMeta, 
ReduceFileMeta, WorkerEventInfo, WorkerInfo, WorkerStatus}
+import org.apache.celeborn.common.meta.{ApplicationMeta, DeviceInfo, 
DiskFileInfo, DiskInfo, MapFileMeta, ReduceFileMeta, WorkerEventInfo, 
WorkerInfo, WorkerStatus}
 import org.apache.celeborn.common.meta.MapFileMeta.SegmentIndex
 import org.apache.celeborn.common.protocol._
 import org.apache.celeborn.common.protocol.PartitionLocation.Mode
@@ -416,45 +415,6 @@ object PbSerDeUtils {
     }.asJava
   }
 
-  def fromPbAppDiskUsage(pbAppDiskUsage: PbAppDiskUsage): AppDiskUsage = {
-    AppDiskUsage(pbAppDiskUsage.getAppId, pbAppDiskUsage.getEstimatedUsage)
-  }
-
-  def toPbAppDiskUsage(appDiskUsage: AppDiskUsage): PbAppDiskUsage = {
-    PbAppDiskUsage.newBuilder()
-      .setAppId(appDiskUsage.appId)
-      .setEstimatedUsage(appDiskUsage.estimatedUsage)
-      .build()
-  }
-
-  def fromPbAppDiskUsageSnapshot(
-      pbAppDiskUsageSnapShot: PbAppDiskUsageSnapshot): AppDiskUsageSnapShot = {
-    val snapShot = new 
AppDiskUsageSnapShot(pbAppDiskUsageSnapShot.getTopItemCount)
-    snapShot.startSnapShotTime = pbAppDiskUsageSnapShot.getStartSnapShotTime
-    snapShot.endSnapShotTime = pbAppDiskUsageSnapShot.getEndSnapshotTime
-    snapShot.restoreFromSnapshot(
-      pbAppDiskUsageSnapShot
-        .getTopNItemsList
-        .asScala
-        .map(fromPbAppDiskUsage)
-        .asJava
-        .stream()
-        .toArray(new IntFunction[Array[AppDiskUsage]]() {
-          override def apply(value: Int): Array[AppDiskUsage] = new 
Array[AppDiskUsage](value)
-        }))
-    snapShot
-  }
-
-  def toPbAppDiskUsageSnapshot(snapshots: AppDiskUsageSnapShot): 
PbAppDiskUsageSnapshot = {
-    PbAppDiskUsageSnapshot.newBuilder()
-      .setTopItemCount(snapshots.topItemCount)
-      .setStartSnapShotTime(snapshots.startSnapShotTime)
-      .setEndSnapshotTime(snapshots.endSnapShotTime)
-      // topNItems some value could be null
-      .addAllTopNItems(snapshots.topNItems.filter(_ != 
null).map(toPbAppDiskUsage).toList.asJava)
-      .build()
-  }
-
   def toPbSnapshotMetaInfo(
       estimatedPartitionSize: java.lang.Long,
       registeredShuffle: java.util.Map[String, java.util.Set[Integer]],
@@ -468,8 +428,6 @@ object PbSerDeUtils {
       partitionTotalFileCount: java.lang.Long,
       shuffleTotalCount: java.lang.Long,
       shuffleFallbackCounts: java.util.Map[String, java.lang.Long],
-      appDiskUsageMetricSnapshots: Array[AppDiskUsageSnapShot],
-      currentAppDiskUsageMetricsSnapshot: AppDiskUsageSnapShot,
       lostWorkers: ConcurrentHashMap[WorkerInfo, java.lang.Long],
       shutdownWorkers: java.util.Set[WorkerInfo],
       workerEventInfos: ConcurrentHashMap[WorkerInfo, WorkerEventInfo],
@@ -491,10 +449,6 @@ object PbSerDeUtils {
       .setPartitionTotalFileCount(partitionTotalFileCount)
       .setShuffleTotalCount(shuffleTotalCount)
       .putAllShuffleFallbackCounts(shuffleFallbackCounts)
-      // appDiskUsageMetricSnapshots can have null values,
-      // protobuf repeated value can't support null value in list.
-      .addAllAppDiskUsageMetricSnapshots(appDiskUsageMetricSnapshots.filter(_ 
!= null)
-        .map(toPbAppDiskUsageSnapshot).toList.asJava)
       .putAllLostWorkers(lostWorkers.asScala.map {
         case (worker: WorkerInfo, time: java.lang.Long) => 
(worker.toUniqueId(), time)
       }.asJava)
@@ -506,10 +460,6 @@ object PbSerDeUtils {
       .addAllDecommissionWorkers(decommissionWorkers.asScala.map(
         toPbWorkerInfo(_, true, false)).asJava)
 
-    if (currentAppDiskUsageMetricsSnapshot != null) {
-      builder.setCurrentAppDiskUsageMetricsSnapshot(
-        toPbAppDiskUsageSnapshot(currentAppDiskUsageMetricsSnapshot))
-    }
     val pbApplicationMetas = applicationMetas.asScala.map {
       case (appId, applicationMeta) => (appId, 
toPbApplicationMeta(applicationMeta))
     }.asJava
diff --git a/docs/configuration/metrics.md b/docs/configuration/metrics.md
index f3140303e..a5fe17318 100644
--- a/docs/configuration/metrics.md
+++ b/docs/configuration/metrics.md
@@ -19,9 +19,6 @@ license: |
 <!--begin-include-->
 | Key | Default | isDynamic | Description | Since | Deprecated |
 | --- | ------- | --------- | ----------- | ----- | ---------- |
-| celeborn.metrics.app.topDiskUsage.count | 50 | false | Size for top items 
about top disk usage applications list. | 0.2.0 |  | 
-| celeborn.metrics.app.topDiskUsage.interval | 10min | false | Time length for 
a window about top disk usage application list. | 0.2.0 |  | 
-| celeborn.metrics.app.topDiskUsage.windowSize | 24 | false | Window size 
about top disk usage application list. | 0.2.0 |  | 
 | celeborn.metrics.capacity | 4096 | false | The maximum number of metrics 
which a source can use to generate output strings. | 0.2.0 |  | 
 | celeborn.metrics.collectPerfCritical.enabled | false | false | It controls 
whether to collect metrics which may affect performance. When enable, Celeborn 
collects them. | 0.2.0 |  | 
 | celeborn.metrics.conf | &lt;undefined&gt; | false | Custom metrics 
configuration file path. Default use `metrics.properties` in classpath. | 0.3.0 
|  | 
diff --git a/docs/developers/sbt.md b/docs/developers/sbt.md
index 181e3b897..0bca08f37 100644
--- a/docs/developers/sbt.md
+++ b/docs/developers/sbt.md
@@ -203,16 +203,16 @@ If you’d like to run just a single Java test in the 
`SlotsAllocatorSuiteJ`, e.
 > testOnly *SlotsAllocatorSuiteJ -- 
 > *SlotsAllocatorSuiteJ.testAllocateSlotsForSinglePartitionId
 ```
 
-If you’d like to run just a single Scala test in the 
`AppDiskUsageMetricSuite`, e.g., a test that includes "app usage snapshot" in 
the name, you run the following command in the sbt console:
+If you’d like to run just a single Scala test in the `MasterSuite`, e.g., a 
test that includes "test single node startup functionality" in the name, you 
run the following command in the sbt console:
 
 ```
-> testOnly *AppDiskUsageMetricSuite -- -z "app usage snapshot"
+> testOnly *MasterSuite -- -z "test single node startup functionality"
 ```
 
 If you’d prefer, you can run all of these commands on the command line (but 
this will be slower than running tests using an open console). To do this, you 
need to surround `testOnly` and the following arguments in quotes:
 
 ```
-$ ./build/sbt "celeborn-master/testOnly *AppDiskUsageMetricSuite -- -z \"app 
usage snapshot\""
+$ ./build/sbt "celeborn-master/testOnly *MasterSuite -- -z \"test single node 
startup functionality\""
 ```
 For more about how to run individual tests with sbt, see the [sbt 
documentation](https://www.scala-sbt.org/1.x/docs/Testing.html) and [JUnit 
Interface](https://github.com/sbt/junit-interface/#junit-interface).
 
diff --git a/docs/migration.md b/docs/migration.md
index 728e63a41..286b90948 100644
--- a/docs/migration.md
+++ b/docs/migration.md
@@ -42,7 +42,6 @@ license: |
     | GET /listDynamicConfigs  | GET /api/v1/conf/dynamic                 |    
                                              |
     | GET /threadDump          | GET /api/v1/thread_dump                  |    
                                              |
     | GET /applications        | GET /api/v1/applications                 |    
                                              |
-    | GET /listTopDiskUsedApps | GET /api/v1/applications/top_disk_usages |    
                                              |
     | GET /hostnames           | GET /api/v1/applications/hostnames       |    
                                              |
     | GET /shuffle             | GET /api/v1/shuffles                     |    
                                              |
     | GET /masterGroupInfo     | GET /api/v1/masters                      |    
                                              |
@@ -64,7 +63,6 @@ license: |
     | GET /listDynamicConfigs        | GET /api/v1/conf/dynamic                
 |                                             |
     | GET /threadDump                | GET /api/v1/thread_dump                 
 |                                             |
     | GET /applications              | GET /api/v1/applications                
 |                                             |
-    | GET /listTopDiskUsedApps       | GET 
/api/v1/applications/top_disk_usages |                                          
   |
     | GET /shuffle                   | GET /api/v1/shuffles                    
 |                                             |
     | GET /listPartitionLocationInfo | GET /api/v1/shuffles/partitions         
 |                                             |
     | GET /workerInfo                | GET /api/v1/workers                     
 |                                             |
@@ -74,6 +72,11 @@ license: |
     | GET /unavailablePeers          | GET /api/v1/workers/unavailable_peers   
 |                                             |
     | POST /exit                     | POST /api/v1/workers/exit               
 |                                             |
 
+- Since 0.6.0, the RESTful api `/listTopDiskUsedApps` both in Master and 
Worker has been removed. Please use the following PromQL query instead.
+  ```text
+  topK(50, sum by (applicationId) 
(metrics_diskBytesWritten_Value{role="worker", applicationId!=""}))
+  ```
+
 ## Upgrading from 0.5.0 to 0.5.1
 
 - Since 0.5.1, Celeborn master REST API `/exclude` request uses media type 
`application/x-www-form-urlencoded` instead of `text/plain`.
diff --git a/docs/restapi.md b/docs/restapi.md
index 73c5c620e..da75338b1 100644
--- a/docs/restapi.md
+++ b/docs/restapi.md
@@ -50,7 +50,6 @@ See the [migration guide](migration.md) for API mappings.
 | /help                | GET    |                                              
| List the available API providers of the master.                               
                                                                                
                                                                                
                                                                                
                               |
 | /hostnames           | GET    |                                              
| List all running application's LifecycleManager's hostnames of the cluster.   
                                                                                
                                                                                
                                                                                
                               |
 | /listDynamicConfigs  | GET    | level=${LEVEL} tenant=${TENANT} name=${NAME} 
| List the dynamic configs of the master. The parameter level specifies the 
config level of dynamic configs. The parameter tenant specifies the tenant id 
of TENANT or TENANT_USER level. The parameter name specifies the user name of 
TENANT_USER level. Meanwhile, either none or all of the parameter tenant and 
name are specified for TENANT_USER level. |
-| /listTopDiskUsedApps | GET    |                                              
| List the top disk usage application ids. It will return the top disk usage 
application ids for the cluster.                                                
                                                                                
                                                                                
                                  |
 | /lostWorkers         | GET    |                                              
| List all lost workers of the master.                                          
                                                                                
                                                                                
                                                                                
                               |
 | /masterGroupInfo     | GET    |                                              
| List master group information of the service. It will list all master's 
LEADER, FOLLOWER information.                                                   
                                                                                
                                                                                
                                     |
 | /metrics/prometheus  | GET    |                                              
| List the metrics data in prometheus format of the master. The url path is 
defined by configure `celeborn.metrics.prometheus.path`.                        
                                                                                
                                                                                
                                   |
@@ -75,7 +74,6 @@ See the [migration guide](migration.md) for API mappings.
 | /isDecommissioning         | GET    |                                        
      | Show if the worker is during the process of decommission.               
                                                                                
                                                                                
                                                                                
                                     |
 | /listDynamicConfigs        | GET    | level=${LEVEL} tenant=${TENANT} 
name=${NAME} | List the dynamic configs of the worker. The parameter level 
specifies the config level of dynamic configs. The parameter tenant specifies 
the tenant id of TENANT or TENANT_USER level. The parameter name specifies the 
user name of TENANT_USER level. Meanwhile, either none or all of the parameter 
tenant and name are specified for TENANT_USER level. |
 | /listPartitionLocationInfo | GET    |                                        
      | List all the living PartitionLocation information in that worker.       
                                                                                
                                                                                
                                                                                
                                     |
-| /listTopDiskUsedApps       | GET    |                                        
      | List the top disk usage application ids. It only return application ids 
running in that worker.                                                         
                                                                                
                                                                                
                                     |
 | /metrics/prometheus        | GET    |                                        
      | List the metrics data in prometheus format of the worker. The url path 
is defined by configure `celeborn.metrics.prometheus.path`.                     
                                                                                
                                                                                
                                      |
 | /shuffles                  | GET    |                                        
      | List all the running shuffle keys of the worker. It only return keys of 
shuffles running in that worker.                                                
                                                                                
                                                                                
                                     |
 | /threadDump                | GET    |                                        
      | List the current thread dump of the worker.                             
                                                                                
                                                                                
                                                                                
                                     |
diff --git 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/AbstractMetaManager.java
 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/AbstractMetaManager.java
index 14c6108bb..cf37796bf 100644
--- 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/AbstractMetaManager.java
+++ 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/AbstractMetaManager.java
@@ -25,7 +25,6 @@ import java.nio.file.Files;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.stream.Collectors;
 
@@ -40,8 +39,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.celeborn.common.CelebornConf;
 import org.apache.celeborn.common.identity.UserIdentifier;
-import org.apache.celeborn.common.meta.AppDiskUsageMetric;
-import org.apache.celeborn.common.meta.AppDiskUsageSnapShot;
 import org.apache.celeborn.common.meta.ApplicationMeta;
 import org.apache.celeborn.common.meta.DiskInfo;
 import org.apache.celeborn.common.meta.DiskStatus;
@@ -96,7 +93,6 @@ public abstract class AbstractMetaManager implements 
IMetadataHandler {
   public final LongAdder partitionTotalFileCount = new LongAdder();
   public final LongAdder shuffleTotalCount = new LongAdder();
   public final Map<String, Long> shuffleFallbackCounts = 
JavaUtils.newConcurrentHashMap();
-  public AppDiskUsageMetric appDiskUsageMetric = null;
 
   public final ConcurrentHashMap<String, ApplicationMeta> applicationMetas =
       JavaUtils.newConcurrentHashMap();
@@ -255,7 +251,6 @@ public abstract class AbstractMetaManager implements 
IMetadataHandler {
       int replicatePort,
       Map<String, DiskInfo> disks,
       Map<UserIdentifier, ResourceConsumption> userResourceConsumption,
-      Map<String, Long> estimatedAppDiskUsage,
       long time,
       WorkerStatus workerStatus,
       boolean highWorkload) {
@@ -285,7 +280,6 @@ public abstract class AbstractMetaManager implements 
IMetadataHandler {
       }
     }
 
-    appDiskUsageMetric.update(estimatedAppDiskUsage);
     // If using HDFSONLY mode, workers with empty disks should not be put into 
excluded worker list.
     long unhealthyDiskNum =
         disks.values().stream().filter(s -> 
!s.status().equals(DiskStatus.HEALTHY)).count();
@@ -369,8 +363,6 @@ public abstract class AbstractMetaManager implements 
IMetadataHandler {
                 partitionTotalFileCount.sum(),
                 shuffleTotalCount.sum(),
                 shuffleFallbackCounts,
-                appDiskUsageMetric.snapShots(),
-                appDiskUsageMetric.currentSnapShot().get(),
                 lostWorkers,
                 shutdownWorkers,
                 workerEventInfos,
@@ -471,14 +463,6 @@ public abstract class AbstractMetaManager implements 
IMetadataHandler {
       
partitionTotalFileCount.add(snapshotMetaInfo.getPartitionTotalFileCount());
       shuffleTotalCount.add(snapshotMetaInfo.getShuffleTotalCount());
       addShuffleFallbackCounts(snapshotMetaInfo.getShuffleFallbackCountsMap());
-      appDiskUsageMetric.restoreFromSnapshot(
-          snapshotMetaInfo.getAppDiskUsageMetricSnapshotsList().stream()
-              .map(PbSerDeUtils::fromPbAppDiskUsageSnapshot)
-              .toArray(AppDiskUsageSnapShot[]::new));
-      appDiskUsageMetric.currentSnapShot_$eq(
-          new AtomicReference<AppDiskUsageSnapShot>(
-              PbSerDeUtils.fromPbAppDiskUsageSnapshot(
-                  snapshotMetaInfo.getCurrentAppDiskUsageMetricsSnapshot())));
 
       snapshotMetaInfo
           .getApplicationMetasMap()
diff --git 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/IMetadataHandler.java
 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/IMetadataHandler.java
index 5ad1907c1..c25135634 100644
--- 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/IMetadataHandler.java
+++ 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/IMetadataHandler.java
@@ -67,7 +67,6 @@ public interface IMetadataHandler {
       int replicatePort,
       Map<String, DiskInfo> disks,
       Map<UserIdentifier, ResourceConsumption> userResourceConsumption,
-      Map<String, Long> estimatedAppDiskUsage,
       long time,
       boolean highWorkload,
       WorkerStatus workerStatus,
diff --git 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/SingleMasterMetaManager.java
 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/SingleMasterMetaManager.java
index 4ddf371bf..765beb791 100644
--- 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/SingleMasterMetaManager.java
+++ 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/SingleMasterMetaManager.java
@@ -25,7 +25,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.celeborn.common.CelebornConf;
 import org.apache.celeborn.common.identity.UserIdentifier;
-import org.apache.celeborn.common.meta.AppDiskUsageMetric;
 import org.apache.celeborn.common.meta.ApplicationMeta;
 import org.apache.celeborn.common.meta.DiskInfo;
 import org.apache.celeborn.common.meta.WorkerInfo;
@@ -48,7 +47,6 @@ public class SingleMasterMetaManager extends 
AbstractMetaManager {
     this.initialEstimatedPartitionSize = conf.initialEstimatedPartitionSize();
     this.estimatedPartitionSize = initialEstimatedPartitionSize;
     this.unhealthyDiskRatioThreshold = 
conf.masterExcludeWorkerUnhealthyDiskRatioThreshold();
-    this.appDiskUsageMetric = new AppDiskUsageMetric(conf);
     this.rackResolver = rackResolver;
   }
 
@@ -121,7 +119,6 @@ public class SingleMasterMetaManager extends 
AbstractMetaManager {
       int replicatePort,
       Map<String, DiskInfo> disks,
       Map<UserIdentifier, ResourceConsumption> userResourceConsumption,
-      Map<String, Long> estimatedAppDiskUsage,
       long time,
       boolean highWorkload,
       WorkerStatus workerStatus,
@@ -134,7 +131,6 @@ public class SingleMasterMetaManager extends 
AbstractMetaManager {
         replicatePort,
         disks,
         userResourceConsumption,
-        estimatedAppDiskUsage,
         time,
         workerStatus,
         highWorkload);
diff --git 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAMasterMetaManager.java
 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAMasterMetaManager.java
index 0c48419d6..1b50b5d7a 100644
--- 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAMasterMetaManager.java
+++ 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/HAMasterMetaManager.java
@@ -28,7 +28,6 @@ import org.apache.celeborn.common.CelebornConf;
 import org.apache.celeborn.common.client.MasterClient;
 import org.apache.celeborn.common.exception.CelebornRuntimeException;
 import org.apache.celeborn.common.identity.UserIdentifier;
-import org.apache.celeborn.common.meta.AppDiskUsageMetric;
 import org.apache.celeborn.common.meta.ApplicationMeta;
 import org.apache.celeborn.common.meta.DiskInfo;
 import org.apache.celeborn.common.meta.WorkerInfo;
@@ -57,7 +56,6 @@ public class HAMasterMetaManager extends AbstractMetaManager {
     this.initialEstimatedPartitionSize = conf.initialEstimatedPartitionSize();
     this.estimatedPartitionSize = initialEstimatedPartitionSize;
     this.unhealthyDiskRatioThreshold = 
conf.masterExcludeWorkerUnhealthyDiskRatioThreshold();
-    this.appDiskUsageMetric = new AppDiskUsageMetric(conf);
     this.rackResolver = rackResolver;
   }
 
@@ -275,7 +273,6 @@ public class HAMasterMetaManager extends 
AbstractMetaManager {
       int replicatePort,
       Map<String, DiskInfo> disks,
       Map<UserIdentifier, ResourceConsumption> userResourceConsumption,
-      Map<String, Long> estimatedAppDiskUsage,
       long time,
       boolean highWorkload,
       WorkerStatus workerStatus,
@@ -295,7 +292,6 @@ public class HAMasterMetaManager extends 
AbstractMetaManager {
                       .putAllDisks(MetaUtil.toPbDiskInfos(disks))
                       .putAllUserResourceConsumption(
                           
MetaUtil.toPbUserResourceConsumption(userResourceConsumption))
-                      .putAllEstimatedAppDiskUsage(estimatedAppDiskUsage)
                       .setWorkerStatus(MetaUtil.toPbWorkerStatus(workerStatus))
                       .setTime(time)
                       .setHighWorkload(highWorkload)
diff --git 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MetaHandler.java
 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MetaHandler.java
index e795c09e5..b1ee9b72e 100644
--- 
a/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MetaHandler.java
+++ 
b/master/src/main/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MetaHandler.java
@@ -101,7 +101,6 @@ public class MetaHandler {
       int replicatePort;
       Map<String, DiskInfo> diskInfos;
       Map<UserIdentifier, ResourceConsumption> userResourceConsumption;
-      Map<String, Long> estimatedAppDiskUsage = new HashMap<>();
       WorkerStatus workerStatus;
       List<Integer> lostShuffles;
       switch (cmdType) {
@@ -203,8 +202,6 @@ public class MetaHandler {
           userResourceConsumption =
               MetaUtil.fromPbUserResourceConsumption(
                   
request.getWorkerHeartbeatRequest().getUserResourceConsumptionMap());
-          estimatedAppDiskUsage.putAll(
-              
request.getWorkerHeartbeatRequest().getEstimatedAppDiskUsageMap());
           replicatePort = 
request.getWorkerHeartbeatRequest().getReplicatePort();
           boolean highWorkload = 
request.getWorkerHeartbeatRequest().getHighWorkload();
           if (request.getWorkerHeartbeatRequest().hasWorkerStatus()) {
@@ -231,7 +228,6 @@ public class MetaHandler {
               replicatePort,
               diskInfos,
               userResourceConsumption,
-              estimatedAppDiskUsage,
               request.getWorkerHeartbeatRequest().getTime(),
               workerStatus,
               highWorkload);
diff --git a/master/src/main/proto/Resource.proto 
b/master/src/main/proto/Resource.proto
index 52b418912..cd9639b3e 100644
--- a/master/src/main/proto/Resource.proto
+++ b/master/src/main/proto/Resource.proto
@@ -158,7 +158,7 @@ message WorkerHeartbeatRequest {
   map<string, DiskInfo> disks = 6;
   required int64 time = 7;
   map<string, ResourceConsumption> userResourceConsumption = 8;
-  map<string, int64> estimatedAppDiskUsage = 9;
+  map<string, int64> estimatedAppDiskUsage = 9; // deprecated
   required bool highWorkload = 10;
   optional WorkerStatus workerStatus = 11;
 }
diff --git 
a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala 
b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala
index f394ab3e0..2859ce3c5 100644
--- 
a/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala
+++ 
b/master/src/main/scala/org/apache/celeborn/service/deploy/master/Master.scala
@@ -494,7 +494,6 @@ private[celeborn] class Master(
           disks,
           userResourceConsumption,
           activeShuffleKey,
-          estimatedAppDiskUsage,
           highWorkload,
           workerStatus,
           requestId) =>
@@ -512,7 +511,6 @@ private[celeborn] class Master(
           disks,
           userResourceConsumption,
           activeShuffleKey,
-          estimatedAppDiskUsage,
           highWorkload,
           workerStatus,
           requestId))
@@ -651,7 +649,6 @@ private[celeborn] class Master(
       disks: Seq[DiskInfo],
       userResourceConsumption: util.Map[UserIdentifier, ResourceConsumption],
       activeShuffleKeys: util.Set[String],
-      estimatedAppDiskUsage: util.HashMap[String, java.lang.Long],
       highWorkload: Boolean,
       workerStatus: WorkerStatus,
       requestId: String): Unit = {
@@ -669,7 +666,6 @@ private[celeborn] class Master(
         replicatePort,
         disks.map { disk => disk.mountPoint -> disk }.toMap.asJava,
         userResourceConsumption,
-        estimatedAppDiskUsage,
         System.currentTimeMillis(),
         highWorkload,
         workerStatus,
@@ -1386,13 +1382,6 @@ private[celeborn] class Master(
     sb.toString()
   }
 
-  override def listTopDiskUseApps: String = {
-    val sb = new StringBuilder
-    sb.append("================== Top Disk Usage Applications 
=======================\n")
-    sb.append(statusSystem.appDiskUsageMetric.summary())
-    sb.toString()
-  }
-
   override def exclude(
       addWorkers: Seq[WorkerInfo],
       removeWorkers: Seq[WorkerInfo]): HandleResponse = {
diff --git 
a/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApplicationResource.scala
 
b/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApplicationResource.scala
index c3ad1e750..0cfa5f4b6 100644
--- 
a/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApplicationResource.scala
+++ 
b/master/src/main/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApplicationResource.scala
@@ -26,8 +26,7 @@ import io.swagger.v3.oas.annotations.media.{Content, Schema}
 import io.swagger.v3.oas.annotations.responses.ApiResponse
 import io.swagger.v3.oas.annotations.tags.Tag
 
-import org.apache.celeborn.common.util.Utils
-import org.apache.celeborn.rest.v1.model.{AppDiskUsageData, 
AppDiskUsageSnapshotData, AppDiskUsageSnapshotsResponse, 
ApplicationHeartbeatData, ApplicationsHeartbeatResponse, DeleteAppsRequest, 
HandleResponse, HostnamesResponse, ReviseLostShufflesRequest}
+import org.apache.celeborn.rest.v1.model.{ApplicationHeartbeatData, 
ApplicationsHeartbeatResponse, DeleteAppsRequest, HandleResponse, 
HostnamesResponse, ReviseLostShufflesRequest}
 import org.apache.celeborn.server.common.http.api.ApiRequestContext
 import org.apache.celeborn.service.deploy.master.Master
 
@@ -67,34 +66,6 @@ class ApplicationResource extends ApiRequestContext {
     new HandleResponse().success(true).message(s"deleted shuffles of app 
${apps}")
   }
 
-  @ApiResponse(
-    responseCode = "200",
-    content = Array(new Content(
-      mediaType = MediaType.APPLICATION_JSON,
-      schema = new Schema(implementation = 
classOf[AppDiskUsageSnapshotsResponse]))),
-    description =
-      "List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster.")
-  @GET
-  @Path("/top_disk_usages")
-  def topDiskUsedApplications(): AppDiskUsageSnapshotsResponse = {
-    new AppDiskUsageSnapshotsResponse()
-      .snapshots(
-        statusSystem.appDiskUsageMetric.topSnapshots().map { snapshot =>
-          new AppDiskUsageSnapshotData()
-            .start(
-              snapshot.startSnapShotTime)
-            .end(
-              snapshot.endSnapShotTime)
-            .topNItems(
-              snapshot.topNItems.filter(_ != null).map { usage =>
-                new AppDiskUsageData()
-                  .appId(usage.appId)
-                  .estimatedUsage(usage.estimatedUsage)
-                  .estimatedUsageStr(Utils.bytesToString(usage.estimatedUsage))
-              }.toSeq.asJava)
-        }.asJava)
-  }
-
   @ApiResponse(
     responseCode = "200",
     content = Array(new Content(
diff --git 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/DefaultMetaSystemSuiteJ.java
 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/DefaultMetaSystemSuiteJ.java
index 3f4c8d064..78695460b 100644
--- 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/DefaultMetaSystemSuiteJ.java
+++ 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/DefaultMetaSystemSuiteJ.java
@@ -697,7 +697,6 @@ public class DefaultMetaSystemSuiteJ {
         REPLICATEPORT1,
         new HashMap<>(),
         userResourceConsumption1,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -714,7 +713,6 @@ public class DefaultMetaSystemSuiteJ {
         REPLICATEPORT2,
         new HashMap<>(),
         userResourceConsumption2,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -731,7 +729,6 @@ public class DefaultMetaSystemSuiteJ {
         REPLICATEPORT3,
         disks3,
         userResourceConsumption3,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -748,7 +745,6 @@ public class DefaultMetaSystemSuiteJ {
         REPLICATEPORT3,
         disks3,
         userResourceConsumption3,
-        new HashMap<>(),
         1,
         true,
         workerStatus,
diff --git 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterStateMachineSuiteJ.java
 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterStateMachineSuiteJ.java
index 1b8b57d74..2b03afd56 100644
--- 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterStateMachineSuiteJ.java
+++ 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/MasterStateMachineSuiteJ.java
@@ -38,7 +38,6 @@ import org.mockito.Mockito;
 import org.apache.celeborn.common.CelebornConf;
 import org.apache.celeborn.common.client.MasterClient;
 import org.apache.celeborn.common.identity.UserIdentifier;
-import org.apache.celeborn.common.meta.AppDiskUsageSnapShot;
 import org.apache.celeborn.common.meta.DiskInfo;
 import org.apache.celeborn.common.meta.WorkerInfo;
 import org.apache.celeborn.common.quota.ResourceConsumption;
@@ -215,22 +214,6 @@ public class MasterStateMachineSuiteJ extends 
RatisBaseSuiteJ {
     masterStatusSystem.hostnameSet.add(host2);
     masterStatusSystem.hostnameSet.add(host3);
 
-    // Wait for update snapshot
-    Thread.sleep(60000);
-    Map<String, Long> appDiskUsage = JavaUtils.newConcurrentHashMap();
-    appDiskUsage.put("app-1", 100L);
-    appDiskUsage.put("app-2", 200L);
-    masterStatusSystem.appDiskUsageMetric.update(appDiskUsage);
-    appDiskUsage.put("app-3", 300L);
-    appDiskUsage.put("app-1", 200L);
-    masterStatusSystem.appDiskUsageMetric.update(appDiskUsage);
-    // wait for snapshot updated
-    Thread.sleep(3000);
-
-    AppDiskUsageSnapShot[] originSnapshots = 
masterStatusSystem.appDiskUsageMetric.snapShots();
-    AppDiskUsageSnapShot originCurrentSnapshot =
-        masterStatusSystem.appDiskUsageMetric.currentSnapShot().get();
-
     WorkerInfo workerInfo1 = new WorkerInfo(host1, 9095, 9094, 9093, 9092, 
9091);
     WorkerInfo workerInfo2 = new WorkerInfo(host2, 9095, 9094, 9093, 9092, 
9091);
     WorkerInfo workerInfo3 = new WorkerInfo(host3, 9095, 9094, 9093, 9092, 
9091);
@@ -253,15 +236,6 @@ public class MasterStateMachineSuiteJ extends 
RatisBaseSuiteJ {
     Assert.assertEquals(3, masterStatusSystem.excludedWorkers.size());
     Assert.assertEquals(2, masterStatusSystem.manuallyExcludedWorkers.size());
     Assert.assertEquals(3, masterStatusSystem.hostnameSet.size());
-    Assert.assertEquals(
-        conf.metricsAppTopDiskUsageWindowSize(),
-        masterStatusSystem.appDiskUsageMetric.snapShots().length);
-    Assert.assertEquals(
-        conf.metricsAppTopDiskUsageCount(),
-        
masterStatusSystem.appDiskUsageMetric.currentSnapShot().get().topNItems().length);
-    Assert.assertEquals(
-        originCurrentSnapshot, 
masterStatusSystem.appDiskUsageMetric.currentSnapShot().get());
-    Assert.assertArrayEquals(originSnapshots, 
masterStatusSystem.appDiskUsageMetric.snapShots());
 
     masterStatusSystem.restoreMetaFromFile(tmpFile);
     Assert.assertEquals(3, masterStatusSystem.workersMap.size());
diff --git 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java
 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java
index 2d6d413cb..0f7f752cb 100644
--- 
a/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java
+++ 
b/master/src/test/java/org/apache/celeborn/service/deploy/master/clustermeta/ha/RatisMasterStatusSystemSuiteJ.java
@@ -1030,7 +1030,6 @@ public class RatisMasterStatusSystemSuiteJ {
         REPLICATEPORT1,
         new HashMap<>(),
         userResourceConsumption1,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -1057,7 +1056,6 @@ public class RatisMasterStatusSystemSuiteJ {
         REPLICATEPORT2,
         new HashMap<>(),
         userResourceConsumption2,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -1087,7 +1085,6 @@ public class RatisMasterStatusSystemSuiteJ {
         REPLICATEPORT1,
         disks1,
         userResourceConsumption1,
-        new HashMap<>(),
         1,
         false,
         workerStatus,
@@ -1117,7 +1114,6 @@ public class RatisMasterStatusSystemSuiteJ {
         REPLICATEPORT1,
         disks1,
         userResourceConsumption1,
-        new HashMap<>(),
         1,
         true,
         workerStatus,
diff --git 
a/master/src/test/scala/org/apache/celeborn/service/deploy/master/AppDiskUsageMetricSuite.scala
 
b/master/src/test/scala/org/apache/celeborn/service/deploy/master/AppDiskUsageMetricSuite.scala
deleted file mode 100644
index f0820dd43..000000000
--- 
a/master/src/test/scala/org/apache/celeborn/service/deploy/master/AppDiskUsageMetricSuite.scala
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.celeborn.service.deploy.master
-
-import java.util
-
-import scala.util.Random
-
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-import org.scalatest.funsuite.AnyFunSuite
-
-import org.apache.celeborn.common.CelebornConf
-import org.apache.celeborn.common.internal.Logging
-import org.apache.celeborn.common.meta.{AppDiskUsageMetric, 
AppDiskUsageSnapShot, WorkerInfo}
-
-class AppDiskUsageMetricSuite extends AnyFunSuite
-  with BeforeAndAfterAll
-  with BeforeAndAfterEach
-  with Logging {
-  val WORKER1 = new WorkerInfo("host1", 111, 112, 113, 114, 115)
-  val WORKER2 = new WorkerInfo("host2", 211, 212, 213, 214, 215)
-  val WORKER3 = new WorkerInfo("host3", 311, 312, 313, 314, 315)
-
-  def verifySnapShotOutput(snapShot: AppDiskUsageSnapShot, capacity: Int, 
appCount: Int): Unit = {
-    val topNItemsEstimatedUsage = snapShot.topNItems
-      .filter(usage => usage != null)
-      .map(_.estimatedUsage)
-
-    assert(snapShot.topItemCount == capacity)
-    assert(topNItemsEstimatedUsage.length == appCount)
-    assert(topNItemsEstimatedUsage sameElements 
topNItemsEstimatedUsage.sorted.reverse)
-  }
-
-  test("test snapshot ordering") {
-    val snapShot = new AppDiskUsageSnapShot(50)
-    val rand = new Random()
-    for (i <- 1 to 5) {
-      snapShot.updateAppDiskUsage(s"app-${i}", rand.nextInt(100000000) + 1)
-    }
-
-    verifySnapShotOutput(snapShot, 50, 5)
-  }
-
-  test("test snapshot ordering with capacity") {
-    val snapShot = new AppDiskUsageSnapShot(50)
-    val rand = new Random()
-    for (i <- 1 to 60) {
-      snapShot.updateAppDiskUsage(s"app-${i}", rand.nextInt(100000000) + 1)
-    }
-
-    verifySnapShotOutput(snapShot, 50, 50)
-  }
-
-  test("test snapshot ordering with duplicate entries") {
-    val snapShot = new AppDiskUsageSnapShot(50)
-    val rand = new Random()
-    for (i <- 1 to 10) {
-      snapShot.updateAppDiskUsage(s"app-${i}", rand.nextInt(100000000) + 1)
-    }
-    for (i <- 1 to 10) {
-      snapShot.updateAppDiskUsage(s"app-${i}", rand.nextInt(100000000) + 
1000000000)
-    }
-
-    verifySnapShotOutput(snapShot, 50, 10)
-  }
-
-  test("test app usage snapshot") {
-    Thread.sleep(5000)
-
-    val conf = new CelebornConf()
-    conf.set(CelebornConf.METRICS_APP_TOP_DISK_USAGE_WINDOW_SIZE.key, "5")
-    conf.set(CelebornConf.METRICS_APP_TOP_DISK_USAGE_INTERVAL.key, "2s")
-    val usageMetric = new AppDiskUsageMetric(conf)
-
-    val map1 = new util.HashMap[String, java.lang.Long]()
-    map1.put("app1", 2874371)
-    map1.put("app2", 43452)
-    map1.put("app3", 2134526)
-    map1.put("app4", 23465463)
-    map1.put("app5", 132456)
-    map1.put("app6", 6535635)
-    usageMetric.update(map1)
-    println(usageMetric.summary())
-    Thread.sleep(2000)
-
-    map1.clear()
-    map1.put("app1", 374524)
-    map1.put("app2", 5234665)
-    map1.put("app3", 24453)
-    map1.put("app4", 2345637)
-    map1.put("app5", 4534)
-    map1.put("app6", 5357)
-    usageMetric.update(map1)
-    println(usageMetric.summary())
-    Thread.sleep(2000)
-
-    map1.clear()
-    map1.put("app1", 12343)
-    map1.put("app2", 3456565)
-    map1.put("app3", 4345)
-    map1.put("app4", 35245268)
-    map1.put("app5", 45367)
-    map1.put("app6", 64345)
-    usageMetric.update(map1)
-    println(usageMetric.summary())
-    Thread.sleep(2500)
-    println(usageMetric.summary())
-  }
-}
diff --git 
a/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApiV1MasterResourceSuite.scala
 
b/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApiV1MasterResourceSuite.scala
index 089e28df1..edcec11c0 100644
--- 
a/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApiV1MasterResourceSuite.scala
+++ 
b/master/src/test/scala/org/apache/celeborn/service/deploy/master/http/api/v1/ApiV1MasterResourceSuite.scala
@@ -25,7 +25,7 @@ import javax.ws.rs.core.MediaType
 
 import org.apache.celeborn.common.CelebornConf
 import org.apache.celeborn.common.util.{CelebornExitKind, ThreadUtils, Utils}
-import org.apache.celeborn.rest.v1.model.{AppDiskUsageSnapshotsResponse, 
ApplicationsResponse, ExcludeWorkerRequest, HandleResponse, HostnamesResponse, 
RemoveWorkersUnavailableInfoRequest, SendWorkerEventRequest, ShufflesResponse, 
WorkerEventsResponse, WorkerId, WorkersResponse}
+import org.apache.celeborn.rest.v1.model.{ApplicationsResponse, 
ExcludeWorkerRequest, HandleResponse, HostnamesResponse, 
RemoveWorkersUnavailableInfoRequest, SendWorkerEventRequest, ShufflesResponse, 
WorkerEventsResponse, WorkerId, WorkersResponse}
 import org.apache.celeborn.server.common.HttpService
 import org.apache.celeborn.server.common.http.api.v1.ApiV1BaseResourceSuite
 import org.apache.celeborn.service.deploy.master.{Master, MasterArguments}
@@ -81,11 +81,6 @@ class ApiV1MasterResourceSuite extends 
ApiV1BaseResourceSuite {
     assert(HttpServletResponse.SC_OK == response.getStatus)
     
assert(response.readEntity(classOf[ApplicationsResponse]).getApplications.isEmpty)
 
-    response =
-      
webTarget.path("applications/top_disk_usages").request(MediaType.APPLICATION_JSON).get()
-    assert(HttpServletResponse.SC_OK == response.getStatus)
-    
assert(response.readEntity(classOf[AppDiskUsageSnapshotsResponse]).getSnapshots.isEmpty)
-
     response = 
webTarget.path("applications/hostnames").request(MediaType.APPLICATION_JSON).get()
     assert(HttpServletResponse.SC_OK == response.getStatus)
     
assert(response.readEntity(classOf[HostnamesResponse]).getHostnames.isEmpty)
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/master/ApplicationApi.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/master/ApplicationApi.java
index dc844dac2..143610b21 100644
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/master/ApplicationApi.java
+++ 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/master/ApplicationApi.java
@@ -25,7 +25,6 @@ import org.apache.celeborn.rest.v1.master.invoker.BaseApi;
 import org.apache.celeborn.rest.v1.master.invoker.Configuration;
 import org.apache.celeborn.rest.v1.master.invoker.Pair;
 
-import org.apache.celeborn.rest.v1.model.AppDiskUsageSnapshotsResponse;
 import org.apache.celeborn.rest.v1.model.ApplicationsHeartbeatResponse;
 import org.apache.celeborn.rest.v1.model.DeleteAppsRequest;
 import org.apache.celeborn.rest.v1.model.HandleResponse;
@@ -254,73 +253,6 @@ public class ApplicationApi extends BaseApi {
     );
   }
 
-  /**
-   * 
-   * List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster. 
-   * @return AppDiskUsageSnapshotsResponse
-   * @throws ApiException if fails to make API call
-   */
-  public AppDiskUsageSnapshotsResponse getApplicationsDiskUsageSnapshots() 
throws ApiException {
-    return this.getApplicationsDiskUsageSnapshots(Collections.emptyMap());
-  }
-
-
-  /**
-   * 
-   * List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster. 
-   * @param additionalHeaders additionalHeaders for this call
-   * @return AppDiskUsageSnapshotsResponse
-   * @throws ApiException if fails to make API call
-   */
-  public AppDiskUsageSnapshotsResponse 
getApplicationsDiskUsageSnapshots(Map<String, String> additionalHeaders) throws 
ApiException {
-    Object localVarPostBody = null;
-    
-    // create path and map variables
-    String localVarPath = "/api/v1/applications/top_disk_usages";
-
-    StringJoiner localVarQueryStringJoiner = new StringJoiner("&");
-    String localVarQueryParameterBaseName;
-    List<Pair> localVarQueryParams = new ArrayList<Pair>();
-    List<Pair> localVarCollectionQueryParams = new ArrayList<Pair>();
-    Map<String, String> localVarHeaderParams = new HashMap<String, String>();
-    Map<String, String> localVarCookieParams = new HashMap<String, String>();
-    Map<String, Object> localVarFormParams = new HashMap<String, Object>();
-
-    
-    localVarHeaderParams.putAll(additionalHeaders);
-
-    
-    
-    final String[] localVarAccepts = {
-      "application/json"
-    };
-    final String localVarAccept = 
apiClient.selectHeaderAccept(localVarAccepts);
-
-    final String[] localVarContentTypes = {
-      
-    };
-    final String localVarContentType = 
apiClient.selectHeaderContentType(localVarContentTypes);
-
-    String[] localVarAuthNames = new String[] { "basic" };
-
-    TypeReference<AppDiskUsageSnapshotsResponse> localVarReturnType = new 
TypeReference<AppDiskUsageSnapshotsResponse>() {};
-    return apiClient.invokeAPI(
-        localVarPath,
-        "GET",
-        localVarQueryParams,
-        localVarCollectionQueryParams,
-        localVarQueryStringJoiner.toString(),
-        localVarPostBody,
-        localVarHeaderParams,
-        localVarCookieParams,
-        localVarFormParams,
-        localVarAccept,
-        localVarContentType,
-        localVarAuthNames,
-        localVarReturnType
-    );
-  }
-
   /**
    * 
    * Revise lost shuffles or deleted shuffles of an application.
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageData.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageData.java
deleted file mode 100644
index 466c4ecaf..000000000
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageData.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.celeborn.rest.v1.model;
-
-import java.util.Objects;
-import java.util.Arrays;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.annotation.JsonValue;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-/**
- * AppDiskUsageData
- */
-@JsonPropertyOrder({
-  AppDiskUsageData.JSON_PROPERTY_APP_ID,
-  AppDiskUsageData.JSON_PROPERTY_ESTIMATED_USAGE,
-  AppDiskUsageData.JSON_PROPERTY_ESTIMATED_USAGE_STR
-})
[email protected](value = 
"org.openapitools.codegen.languages.JavaClientCodegen", comments = "Generator 
version: 7.8.0")
-public class AppDiskUsageData {
-  public static final String JSON_PROPERTY_APP_ID = "appId";
-  private String appId;
-
-  public static final String JSON_PROPERTY_ESTIMATED_USAGE = "estimatedUsage";
-  private Long estimatedUsage;
-
-  public static final String JSON_PROPERTY_ESTIMATED_USAGE_STR = 
"estimatedUsageStr";
-  private String estimatedUsageStr;
-
-  public AppDiskUsageData() {
-  }
-
-  public AppDiskUsageData appId(String appId) {
-    
-    this.appId = appId;
-    return this;
-  }
-
-  /**
-   * The id of the application.
-   * @return appId
-   */
-  @javax.annotation.Nonnull
-  @JsonProperty(JSON_PROPERTY_APP_ID)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-
-  public String getAppId() {
-    return appId;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_APP_ID)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-  public void setAppId(String appId) {
-    this.appId = appId;
-  }
-
-  public AppDiskUsageData estimatedUsage(Long estimatedUsage) {
-    
-    this.estimatedUsage = estimatedUsage;
-    return this;
-  }
-
-  /**
-   * The application disk usage.
-   * @return estimatedUsage
-   */
-  @javax.annotation.Nonnull
-  @JsonProperty(JSON_PROPERTY_ESTIMATED_USAGE)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-
-  public Long getEstimatedUsage() {
-    return estimatedUsage;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_ESTIMATED_USAGE)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-  public void setEstimatedUsage(Long estimatedUsage) {
-    this.estimatedUsage = estimatedUsage;
-  }
-
-  public AppDiskUsageData estimatedUsageStr(String estimatedUsageStr) {
-    
-    this.estimatedUsageStr = estimatedUsageStr;
-    return this;
-  }
-
-  /**
-   * The application disk usage in string type.
-   * @return estimatedUsageStr
-   */
-  @javax.annotation.Nonnull
-  @JsonProperty(JSON_PROPERTY_ESTIMATED_USAGE_STR)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-
-  public String getEstimatedUsageStr() {
-    return estimatedUsageStr;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_ESTIMATED_USAGE_STR)
-  @JsonInclude(value = JsonInclude.Include.ALWAYS)
-  public void setEstimatedUsageStr(String estimatedUsageStr) {
-    this.estimatedUsageStr = estimatedUsageStr;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    AppDiskUsageData appDiskUsageData = (AppDiskUsageData) o;
-    return Objects.equals(this.appId, appDiskUsageData.appId) &&
-        Objects.equals(this.estimatedUsage, appDiskUsageData.estimatedUsage) &&
-        Objects.equals(this.estimatedUsageStr, 
appDiskUsageData.estimatedUsageStr);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(appId, estimatedUsage, estimatedUsageStr);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("class AppDiskUsageData {\n");
-    sb.append("    appId: ").append(toIndentedString(appId)).append("\n");
-    sb.append("    estimatedUsage: 
").append(toIndentedString(estimatedUsage)).append("\n");
-    sb.append("    estimatedUsageStr: 
").append(toIndentedString(estimatedUsageStr)).append("\n");
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * Convert the given object to string with each line indented by 4 spaces
-   * (except the first line).
-   */
-  private String toIndentedString(Object o) {
-    if (o == null) {
-      return "null";
-    }
-    return o.toString().replace("\n", "\n    ");
-  }
-
-}
-
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotData.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotData.java
deleted file mode 100644
index 4f91a3905..000000000
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotData.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * 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.celeborn.rest.v1.model;
-
-import java.util.Objects;
-import java.util.Arrays;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.annotation.JsonValue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.celeborn.rest.v1.model.AppDiskUsageData;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-/**
- * AppDiskUsageSnapshotData
- */
-@JsonPropertyOrder({
-  AppDiskUsageSnapshotData.JSON_PROPERTY_START,
-  AppDiskUsageSnapshotData.JSON_PROPERTY_END,
-  AppDiskUsageSnapshotData.JSON_PROPERTY_TOP_N_ITEMS
-})
[email protected](value = 
"org.openapitools.codegen.languages.JavaClientCodegen", comments = "Generator 
version: 7.8.0")
-public class AppDiskUsageSnapshotData {
-  public static final String JSON_PROPERTY_START = "start";
-  private Long start;
-
-  public static final String JSON_PROPERTY_END = "end";
-  private Long end;
-
-  public static final String JSON_PROPERTY_TOP_N_ITEMS = "topNItems";
-  private List<AppDiskUsageData> topNItems = new ArrayList<>();
-
-  public AppDiskUsageSnapshotData() {
-  }
-
-  public AppDiskUsageSnapshotData start(Long start) {
-    
-    this.start = start;
-    return this;
-  }
-
-  /**
-   * The start timestamp of the snapshot.
-   * @return start
-   */
-  @javax.annotation.Nullable
-  @JsonProperty(JSON_PROPERTY_START)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-
-  public Long getStart() {
-    return start;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_START)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-  public void setStart(Long start) {
-    this.start = start;
-  }
-
-  public AppDiskUsageSnapshotData end(Long end) {
-    
-    this.end = end;
-    return this;
-  }
-
-  /**
-   * The end timestamp of the snapshot.
-   * @return end
-   */
-  @javax.annotation.Nullable
-  @JsonProperty(JSON_PROPERTY_END)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-
-  public Long getEnd() {
-    return end;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_END)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-  public void setEnd(Long end) {
-    this.end = end;
-  }
-
-  public AppDiskUsageSnapshotData topNItems(List<AppDiskUsageData> topNItems) {
-    
-    this.topNItems = topNItems;
-    return this;
-  }
-
-  public AppDiskUsageSnapshotData addTopNItemsItem(AppDiskUsageData 
topNItemsItem) {
-    if (this.topNItems == null) {
-      this.topNItems = new ArrayList<>();
-    }
-    this.topNItems.add(topNItemsItem);
-    return this;
-  }
-
-  /**
-   * The top N app disk usages.
-   * @return topNItems
-   */
-  @javax.annotation.Nullable
-  @JsonProperty(JSON_PROPERTY_TOP_N_ITEMS)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-
-  public List<AppDiskUsageData> getTopNItems() {
-    return topNItems;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_TOP_N_ITEMS)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-  public void setTopNItems(List<AppDiskUsageData> topNItems) {
-    this.topNItems = topNItems;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    AppDiskUsageSnapshotData appDiskUsageSnapshotData = 
(AppDiskUsageSnapshotData) o;
-    return Objects.equals(this.start, appDiskUsageSnapshotData.start) &&
-        Objects.equals(this.end, appDiskUsageSnapshotData.end) &&
-        Objects.equals(this.topNItems, appDiskUsageSnapshotData.topNItems);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(start, end, topNItems);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("class AppDiskUsageSnapshotData {\n");
-    sb.append("    start: ").append(toIndentedString(start)).append("\n");
-    sb.append("    end: ").append(toIndentedString(end)).append("\n");
-    sb.append("    topNItems: 
").append(toIndentedString(topNItems)).append("\n");
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * Convert the given object to string with each line indented by 4 spaces
-   * (except the first line).
-   */
-  private String toIndentedString(Object o) {
-    if (o == null) {
-      return "null";
-    }
-    return o.toString().replace("\n", "\n    ");
-  }
-
-}
-
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotsResponse.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotsResponse.java
deleted file mode 100644
index 6e6a416ff..000000000
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsageSnapshotsResponse.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.celeborn.rest.v1.model;
-
-import java.util.Objects;
-import java.util.Arrays;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.annotation.JsonValue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.celeborn.rest.v1.model.AppDiskUsageSnapshotData;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-/**
- * AppDiskUsageSnapshotsResponse
- */
-@JsonPropertyOrder({
-  AppDiskUsageSnapshotsResponse.JSON_PROPERTY_SNAPSHOTS
-})
[email protected](value = 
"org.openapitools.codegen.languages.JavaClientCodegen", comments = "Generator 
version: 7.8.0")
-public class AppDiskUsageSnapshotsResponse {
-  public static final String JSON_PROPERTY_SNAPSHOTS = "snapshots";
-  private List<AppDiskUsageSnapshotData> snapshots = new ArrayList<>();
-
-  public AppDiskUsageSnapshotsResponse() {
-  }
-
-  public AppDiskUsageSnapshotsResponse 
snapshots(List<AppDiskUsageSnapshotData> snapshots) {
-    
-    this.snapshots = snapshots;
-    return this;
-  }
-
-  public AppDiskUsageSnapshotsResponse 
addSnapshotsItem(AppDiskUsageSnapshotData snapshotsItem) {
-    if (this.snapshots == null) {
-      this.snapshots = new ArrayList<>();
-    }
-    this.snapshots.add(snapshotsItem);
-    return this;
-  }
-
-  /**
-   * The disk usage snapshots.
-   * @return snapshots
-   */
-  @javax.annotation.Nullable
-  @JsonProperty(JSON_PROPERTY_SNAPSHOTS)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-
-  public List<AppDiskUsageSnapshotData> getSnapshots() {
-    return snapshots;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_SNAPSHOTS)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-  public void setSnapshots(List<AppDiskUsageSnapshotData> snapshots) {
-    this.snapshots = snapshots;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    AppDiskUsageSnapshotsResponse appDiskUsageSnapshotsResponse = 
(AppDiskUsageSnapshotsResponse) o;
-    return Objects.equals(this.snapshots, 
appDiskUsageSnapshotsResponse.snapshots);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(snapshots);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("class AppDiskUsageSnapshotsResponse {\n");
-    sb.append("    snapshots: 
").append(toIndentedString(snapshots)).append("\n");
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * Convert the given object to string with each line indented by 4 spaces
-   * (except the first line).
-   */
-  private String toIndentedString(Object o) {
-    if (o == null) {
-      return "null";
-    }
-    return o.toString().replace("\n", "\n    ");
-  }
-
-}
-
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsagesResponse.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsagesResponse.java
deleted file mode 100644
index f133b95d5..000000000
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/model/AppDiskUsagesResponse.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.celeborn.rest.v1.model;
-
-import java.util.Objects;
-import java.util.Arrays;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.annotation.JsonValue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.celeborn.rest.v1.model.AppDiskUsageData;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-/**
- * AppDiskUsagesResponse
- */
-@JsonPropertyOrder({
-  AppDiskUsagesResponse.JSON_PROPERTY_APP_DISK_USAGES
-})
[email protected](value = 
"org.openapitools.codegen.languages.JavaClientCodegen", comments = "Generator 
version: 7.8.0")
-public class AppDiskUsagesResponse {
-  public static final String JSON_PROPERTY_APP_DISK_USAGES = "appDiskUsages";
-  private List<AppDiskUsageData> appDiskUsages = new ArrayList<>();
-
-  public AppDiskUsagesResponse() {
-  }
-
-  public AppDiskUsagesResponse appDiskUsages(List<AppDiskUsageData> 
appDiskUsages) {
-    
-    this.appDiskUsages = appDiskUsages;
-    return this;
-  }
-
-  public AppDiskUsagesResponse addAppDiskUsagesItem(AppDiskUsageData 
appDiskUsagesItem) {
-    if (this.appDiskUsages == null) {
-      this.appDiskUsages = new ArrayList<>();
-    }
-    this.appDiskUsages.add(appDiskUsagesItem);
-    return this;
-  }
-
-  /**
-   * The application disk usages.
-   * @return appDiskUsages
-   */
-  @javax.annotation.Nullable
-  @JsonProperty(JSON_PROPERTY_APP_DISK_USAGES)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-
-  public List<AppDiskUsageData> getAppDiskUsages() {
-    return appDiskUsages;
-  }
-
-
-  @JsonProperty(JSON_PROPERTY_APP_DISK_USAGES)
-  @JsonInclude(value = JsonInclude.Include.USE_DEFAULTS)
-  public void setAppDiskUsages(List<AppDiskUsageData> appDiskUsages) {
-    this.appDiskUsages = appDiskUsages;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    AppDiskUsagesResponse appDiskUsagesResponse = (AppDiskUsagesResponse) o;
-    return Objects.equals(this.appDiskUsages, 
appDiskUsagesResponse.appDiskUsages);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(appDiskUsages);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("class AppDiskUsagesResponse {\n");
-    sb.append("    appDiskUsages: 
").append(toIndentedString(appDiskUsages)).append("\n");
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * Convert the given object to string with each line indented by 4 spaces
-   * (except the first line).
-   */
-  private String toIndentedString(Object o) {
-    if (o == null) {
-      return "null";
-    }
-    return o.toString().replace("\n", "\n    ");
-  }
-
-}
-
diff --git 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/worker/ApplicationApi.java
 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/worker/ApplicationApi.java
index d14878cdb..62bb6734f 100644
--- 
a/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/worker/ApplicationApi.java
+++ 
b/openapi/openapi-client/src/main/java/org/apache/celeborn/rest/v1/worker/ApplicationApi.java
@@ -25,7 +25,6 @@ import org.apache.celeborn.rest.v1.worker.invoker.BaseApi;
 import org.apache.celeborn.rest.v1.worker.invoker.Configuration;
 import org.apache.celeborn.rest.v1.worker.invoker.Pair;
 
-import org.apache.celeborn.rest.v1.model.AppDiskUsagesResponse;
 import org.apache.celeborn.rest.v1.model.ApplicationsResponse;
 
 
@@ -114,73 +113,6 @@ public class ApplicationApi extends BaseApi {
     );
   }
 
-  /**
-   * 
-   * List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster. 
-   * @return AppDiskUsagesResponse
-   * @throws ApiException if fails to make API call
-   */
-  public AppDiskUsagesResponse getApplicationsDiskUsage() throws ApiException {
-    return this.getApplicationsDiskUsage(Collections.emptyMap());
-  }
-
-
-  /**
-   * 
-   * List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster. 
-   * @param additionalHeaders additionalHeaders for this call
-   * @return AppDiskUsagesResponse
-   * @throws ApiException if fails to make API call
-   */
-  public AppDiskUsagesResponse getApplicationsDiskUsage(Map<String, String> 
additionalHeaders) throws ApiException {
-    Object localVarPostBody = null;
-    
-    // create path and map variables
-    String localVarPath = "/api/v1/applications/top_disk_usages";
-
-    StringJoiner localVarQueryStringJoiner = new StringJoiner("&");
-    String localVarQueryParameterBaseName;
-    List<Pair> localVarQueryParams = new ArrayList<Pair>();
-    List<Pair> localVarCollectionQueryParams = new ArrayList<Pair>();
-    Map<String, String> localVarHeaderParams = new HashMap<String, String>();
-    Map<String, String> localVarCookieParams = new HashMap<String, String>();
-    Map<String, Object> localVarFormParams = new HashMap<String, Object>();
-
-    
-    localVarHeaderParams.putAll(additionalHeaders);
-
-    
-    
-    final String[] localVarAccepts = {
-      "application/json"
-    };
-    final String localVarAccept = 
apiClient.selectHeaderAccept(localVarAccepts);
-
-    final String[] localVarContentTypes = {
-      
-    };
-    final String localVarContentType = 
apiClient.selectHeaderContentType(localVarContentTypes);
-
-    String[] localVarAuthNames = new String[] { "basic" };
-
-    TypeReference<AppDiskUsagesResponse> localVarReturnType = new 
TypeReference<AppDiskUsagesResponse>() {};
-    return apiClient.invokeAPI(
-        localVarPath,
-        "GET",
-        localVarQueryParams,
-        localVarCollectionQueryParams,
-        localVarQueryStringJoiner.toString(),
-        localVarPostBody,
-        localVarHeaderParams,
-        localVarCookieParams,
-        localVarFormParams,
-        localVarAccept,
-        localVarContentType,
-        localVarAuthNames,
-        localVarReturnType
-    );
-  }
-
   @Override
   public <T> T invokeAPI(String url, String method, Object request, 
TypeReference<T> returnType, Map<String, String> additionalHeaders) throws 
ApiException {
     String localVarPath = url.replace(apiClient.getBaseURL(), "");
diff --git a/openapi/openapi-client/src/main/openapi3/master_rest_v1.yaml 
b/openapi/openapi-client/src/main/openapi3/master_rest_v1.yaml
index 72ab9d31d..62ee5efc9 100644
--- a/openapi/openapi-client/src/main/openapi3/master_rest_v1.yaml
+++ b/openapi/openapi-client/src/main/openapi3/master_rest_v1.yaml
@@ -257,22 +257,6 @@ paths:
               schema:
                 $ref: '#/components/schemas/HandleResponse'
 
-  /api/v1/applications/top_disk_usages:
-    get:
-      tags:
-        - Application
-      operationId: getApplicationsDiskUsageSnapshots
-      description: |
-        List the top disk usage application ids.
-        It will return the top disk usage application ids for the cluster.
-      responses:
-        "200":
-          description: The request was successful.
-          content:
-            application/json:
-              schema:
-                $ref: '#/components/schemas/AppDiskUsageSnapshotsResponse'
-
   /api/v1/applications/hostnames:
     get:
       tags:
@@ -636,51 +620,6 @@ components:
       required:
         - hostnames
 
-    AppDiskUsageData:
-      type: object
-      properties:
-        appId:
-          type: string
-          description: The id of the application.
-        estimatedUsage:
-          type: integer
-          format: int64
-          description: The application disk usage.
-        estimatedUsageStr:
-          type: string
-          description: The application disk usage in string type.
-      required:
-        - appId
-        - estimatedUsage
-        - estimatedUsageStr
-
-
-    AppDiskUsageSnapshotData:
-      type: object
-      properties:
-        start:
-          type: integer
-          format: int64
-          description: The start timestamp of the snapshot.
-        end:
-          type: integer
-          format: int64
-          description: The end timestamp of the snapshot.
-        topNItems:
-          type: array
-          description: The top N app disk usages.
-          items:
-            $ref: '#/components/schemas/AppDiskUsageData'
-
-    AppDiskUsageSnapshotsResponse:
-      type: object
-      properties:
-        snapshots:
-          type: array
-          description: The disk usage snapshots.
-          items:
-            $ref: '#/components/schemas/AppDiskUsageSnapshotData'
-
     MasterLeader:
       type: object
       properties:
diff --git a/openapi/openapi-client/src/main/openapi3/worker_rest_v1.yaml 
b/openapi/openapi-client/src/main/openapi3/worker_rest_v1.yaml
index 52f927db0..630ea5da6 100644
--- a/openapi/openapi-client/src/main/openapi3/worker_rest_v1.yaml
+++ b/openapi/openapi-client/src/main/openapi3/worker_rest_v1.yaml
@@ -202,22 +202,6 @@ paths:
               schema:
                 $ref: '#/components/schemas/ApplicationsResponse'
 
-  /api/v1/applications/top_disk_usages:
-    get:
-      tags:
-        - Application
-      operationId: getApplicationsDiskUsage
-      description: |
-        List the top disk usage application ids.
-        It will return the top disk usage application ids for the cluster.
-      responses:
-        "200":
-          description: The request was successful.
-          content:
-            application/json:
-              schema:
-                $ref: '#/components/schemas/AppDiskUsagesResponse'
-
 components:
   schemas:
     ConfigData:
@@ -357,33 +341,6 @@ components:
       required:
         - applications
 
-    AppDiskUsageData:
-      type: object
-      properties:
-        appId:
-          type: string
-          description: The id of the application.
-        estimatedUsage:
-          type: integer
-          format: int64
-          description: The application disk usage.
-        estimatedUsageStr:
-          type: string
-          description: The application disk usage in string type.
-      required:
-        - appId
-        - estimatedUsage
-        - estimatedUsageStr
-
-    AppDiskUsagesResponse:
-      type: object
-      properties:
-        appDiskUsages:
-          type: array
-          description: The application disk usages.
-          items:
-            $ref: '#/components/schemas/AppDiskUsageData'
-
     WorkerInfoResponse:
       type: object
       properties:
diff --git 
a/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala 
b/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala
index bb98e88f6..b123a0baa 100644
--- a/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala
+++ b/service/src/main/scala/org/apache/celeborn/server/common/HttpService.scala
@@ -158,8 +158,6 @@ abstract class HttpService extends Service with Logging {
 
   def getApplicationList: String
 
-  def listTopDiskUseApps: String
-
   def getMasterGroupInfo: String = throw new UnsupportedOperationException()
 
   def getLostWorkers: String = throw new UnsupportedOperationException()
diff --git 
a/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala
 
b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala
index b9dbecbfe..03dbe69ce 100644
--- 
a/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala
+++ 
b/service/src/main/scala/org/apache/celeborn/server/common/http/api/ApiBaseResource.scala
@@ -113,17 +113,4 @@ private[api] class ApiBaseResource extends 
ApiRequestContext {
   def applications(): String = {
     httpService.getApplicationList
   }
-
-  @Path("listTopDiskUsedApps")
-  @ApiResponse(
-    responseCode = "200",
-    content = Array(new Content(
-      mediaType = MediaType.TEXT_PLAIN)),
-    description =
-      "For MASTER: List the top disk usage application ids. It will return the 
top disk usage application ids for the cluster.\n" +
-        "For WORKER: List the top disk usage application ids. It only return 
application ids running in that worker.")
-  @GET
-  def listTopDiskUsedApps(): String = {
-    httpService.listTopDiskUseApps
-  }
 }
diff --git 
a/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala
 
b/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala
index 4a0c2cbd4..09945217b 100644
--- 
a/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala
+++ 
b/service/src/test/scala/org/apache/celeborn/server/common/http/ApiBaseResourceSuite.scala
@@ -68,11 +68,6 @@ abstract class ApiBaseResourceSuite extends HttpTestHelper {
     assert(HttpServletResponse.SC_OK == response.getStatus)
   }
 
-  test("listTopDiskUsedApps") {
-    val response = 
webTarget.path("listTopDiskUsedApps").request(MediaType.TEXT_PLAIN).get()
-    assert(HttpServletResponse.SC_OK == response.getStatus)
-  }
-
   test("swagger") {
     Seq("swagger", "docs", "help").foreach { path =>
       val response = webTarget.path(path).request(MediaType.TEXT_HTML).get()
diff --git 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala
index 7b8d5731a..4edf64ba0 100644
--- 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala
+++ 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/Worker.scala
@@ -18,9 +18,8 @@
 package org.apache.celeborn.service.deploy.worker
 
 import java.io.File
-import java.lang.{Long => JLong}
 import java.util
-import java.util.{HashMap => JHashMap, HashSet => JHashSet, Locale, Map => 
JMap, UUID}
+import java.util.{HashSet => JHashSet, Locale, Map => JMap, UUID}
 import java.util.concurrent._
 import java.util.concurrent.atomic.{AtomicBoolean, AtomicIntegerArray}
 
@@ -456,12 +455,8 @@ private[celeborn] class Worker(
 
   private def heartbeatToMaster(): Unit = {
     val activeShuffleKeys = new JHashSet[String]()
-    val estimatedAppDiskUsage = new JHashMap[String, JLong]()
     activeShuffleKeys.addAll(partitionLocationInfo.shuffleKeySet)
     activeShuffleKeys.addAll(storageManager.shuffleKeySet())
-    storageManager.topAppDiskUsage(true).asScala.foreach { case (shuffleId, 
usage) =>
-      estimatedAppDiskUsage.put(shuffleId, usage)
-    }
     storageManager.updateDiskInfos()
     val diskInfos =
       workerInfo.updateThenGetDiskInfos(storageManager.disksSnapshot().map { 
disk =>
@@ -478,7 +473,6 @@ private[celeborn] class Worker(
         diskInfos,
         handleResourceConsumption(),
         activeShuffleKeys,
-        estimatedAppDiskUsage,
         highWorkload,
         workerStatusManager.currentWorkerStatus),
       classOf[HeartbeatFromWorkerResponse])
@@ -852,15 +846,6 @@ private[celeborn] class Worker(
     sb.toString()
   }
 
-  override def listTopDiskUseApps: String = {
-    val sb = new StringBuilder
-    sb.append("================== Top Disk Usage Applications 
=======================\n")
-    storageManager.topAppDiskUsage().asScala.foreach { case (appId, usage) =>
-      sb.append(s"Application $appId used ${Utils.bytesToString(usage)}\n")
-    }
-    sb.toString()
-  }
-
   override def listPartitionLocationInfo: String = {
     val sb = new StringBuilder
     sb.append("==================== Partition Location Info 
=========================\n")
diff --git 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApplicationResource.scala
 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApplicationResource.scala
index 2f28506ca..0020f5367 100644
--- 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApplicationResource.scala
+++ 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApplicationResource.scala
@@ -17,7 +17,7 @@
 
 package org.apache.celeborn.service.deploy.worker.http.api.v1
 
-import javax.ws.rs.{Consumes, GET, Path, Produces}
+import javax.ws.rs.{Consumes, GET, Produces}
 import javax.ws.rs.core.MediaType
 
 import scala.collection.JavaConverters._
@@ -26,8 +26,7 @@ import io.swagger.v3.oas.annotations.media.{Content, Schema}
 import io.swagger.v3.oas.annotations.responses.ApiResponse
 import io.swagger.v3.oas.annotations.tags.Tag
 
-import org.apache.celeborn.common.util.Utils
-import org.apache.celeborn.rest.v1.model.{AppDiskUsageData, 
AppDiskUsagesResponse, ApplicationsResponse}
+import org.apache.celeborn.rest.v1.model.ApplicationsResponse
 import org.apache.celeborn.server.common.http.api.ApiRequestContext
 import org.apache.celeborn.service.deploy.worker.Worker
 
@@ -36,7 +35,6 @@ import org.apache.celeborn.service.deploy.worker.Worker
 @Consumes(Array(MediaType.APPLICATION_JSON))
 class ApplicationResource extends ApiRequestContext {
   private def worker = httpService.asInstanceOf[Worker]
-  private def storageManager = worker.storageManager
 
   @ApiResponse(
     responseCode = "200",
@@ -50,25 +48,4 @@ class ApplicationResource extends ApiRequestContext {
     new ApplicationsResponse()
       .applications(worker.workerInfo.getApplicationIdSet.asScala.toSeq.asJava)
   }
-
-  @ApiResponse(
-    responseCode = "200",
-    content = Array(new Content(
-      mediaType = MediaType.APPLICATION_JSON,
-      schema = new Schema(implementation = classOf[AppDiskUsagesResponse]))),
-    description =
-      "List the top disk usage application ids. It will return the top disk 
usage application ids for the cluster.")
-  @GET
-  @Path("/top_disk_usages")
-  def topDiskUsedApplications(): AppDiskUsagesResponse = {
-    new AppDiskUsagesResponse()
-      .appDiskUsages(
-        storageManager.topAppDiskUsage().asScala.map { case (appId, diskUsage) 
=>
-          new AppDiskUsageData()
-            .appId(appId)
-            .estimatedUsage(diskUsage)
-            .estimatedUsageStr(Utils.bytesToString(diskUsage))
-        }.toSeq.asJava)
-
-  }
 }
diff --git 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala
 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala
index 87d6de179..0732b9ced 100644
--- 
a/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala
+++ 
b/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala
@@ -77,8 +77,6 @@ final private[worker] class StorageManager(conf: 
CelebornConf, workerSource: Abs
   val diskReserveSize = conf.workerDiskReserveSize
   val diskReserveRatio = conf.workerDiskReserveRatio
 
-  val topDiskUsageCount = conf.metricsAppTopDiskUsageCount
-
   // (deviceName -> deviceInfo) and (mount point -> diskInfo)
   val (deviceInfos, diskInfos) = {
     val workingDirInfos =
@@ -556,19 +554,6 @@ final private[worker] class StorageManager(conf: 
CelebornConf, workerSource: Abs
     hashSet
   }
 
-  def topAppDiskUsage(reportToMaster: Boolean = false): util.Map[String, Long] 
= {
-    val topCount = if (reportToMaster) topDiskUsageCount * 2 else 
topDiskUsageCount
-    diskFileInfos.asScala.map { keyedWriters =>
-      {
-        keyedWriters._1 -> 
keyedWriters._2.values().asScala.map(_.getFileLength).sum
-      }
-    }.toList.map { case (shuffleKey, usage) =>
-      Utils.splitShuffleKey(shuffleKey)._1 -> usage
-    }.groupBy(_._1).map { case (key, values) =>
-      key -> values.map(_._2).sum
-    }.toSeq.sortBy(_._2).reverse.take(topCount).toMap.asJava
-  }
-
   def cleanFile(shuffleKey: String, fileName: String): Unit = {
     cleanFileInternal(shuffleKey, removeFileInfo(shuffleKey, fileName))
   }
diff --git 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1OpenapiClientSuite.scala
 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1OpenapiClientSuite.scala
index cab8130c0..0c1e31543 100644
--- 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1OpenapiClientSuite.scala
+++ 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1OpenapiClientSuite.scala
@@ -52,7 +52,6 @@ class ApiV1OpenapiClientSuite extends 
ApiV1WorkerOpenapiClientSuite {
     val api = new ApplicationApi(masterApiClient)
     assert(api.getApplications.getApplications.isEmpty)
     assert(api.getApplicationHostNames.getHostnames.isEmpty)
-    assert(api.getApplicationsDiskUsageSnapshots.getSnapshots.isEmpty)
   }
 
   test("master: master api") {
diff --git 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerOpenapiClientSuite.scala
 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerOpenapiClientSuite.scala
index 8b69ca338..0e10e9259 100644
--- 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerOpenapiClientSuite.scala
+++ 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerOpenapiClientSuite.scala
@@ -67,7 +67,6 @@ abstract class ApiV1WorkerOpenapiClientSuite extends 
CelebornFunSuite with MiniC
   test("worker: application api") {
     val api = new ApplicationApi(workerApiClient)
     assert(api.getApplicationList.getApplications.isEmpty)
-    assert(api.getApplicationsDiskUsage.getAppDiskUsages.isEmpty)
   }
 
   test("worker: shuffle api") {
diff --git 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerResourceSuite.scala
 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerResourceSuite.scala
index d30c8f4be..87bc65481 100644
--- 
a/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerResourceSuite.scala
+++ 
b/worker/src/test/scala/org/apache/celeborn/service/deploy/worker/http/api/v1/ApiV1WorkerResourceSuite.scala
@@ -20,7 +20,7 @@ package org.apache.celeborn.service.deploy.worker.http.api.v1
 import javax.servlet.http.HttpServletResponse
 import javax.ws.rs.core.MediaType
 
-import org.apache.celeborn.rest.v1.model.{AppDiskUsagesResponse, 
ApplicationsResponse, ShufflePartitionsResponse, ShufflesResponse, 
UnAvailablePeersResponse, WorkerInfoResponse}
+import org.apache.celeborn.rest.v1.model.{ApplicationsResponse, 
ShufflePartitionsResponse, ShufflesResponse, UnAvailablePeersResponse, 
WorkerInfoResponse}
 import org.apache.celeborn.server.common.HttpService
 import org.apache.celeborn.server.common.http.api.v1.ApiV1BaseResourceSuite
 import org.apache.celeborn.service.deploy.MiniClusterFeature
@@ -60,11 +60,6 @@ class ApiV1WorkerResourceSuite extends 
ApiV1BaseResourceSuite with MiniClusterFe
     var response = 
webTarget.path("applications").request(MediaType.APPLICATION_JSON).get()
     assert(HttpServletResponse.SC_OK == response.getStatus)
     
assert(response.readEntity(classOf[ApplicationsResponse]).getApplications.isEmpty)
-
-    response =
-      
webTarget.path("applications/top_disk_usages").request(MediaType.APPLICATION_JSON).get()
-    assert(HttpServletResponse.SC_OK == response.getStatus)
-    
assert(response.readEntity(classOf[AppDiskUsagesResponse]).getAppDiskUsages.isEmpty)
   }
 
   test("worker resource") {

Reply via email to