Skip to content

Commit 16186cd

Browse files
committed
[SPARK-20955][CORE] Intern "executorId" to reduce the memory usage
## What changes were proposed in this pull request? In [this line](https://github.com/apache/spark/blob/f7cf2096fdecb8edab61c8973c07c6fc877ee32d/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala#L128), it uses the `executorId` string received from executors and finally it will go into `TaskUIData`. As deserializing the `executorId` string will always create a new instance, we have a lot of duplicated string instances. This PR does a String interning for TaskUIData to reduce the memory usage. ## How was this patch tested? Manually test using `bin/spark-shell --master local-cluster[6,1,1024]`. Test codes: ``` for (_ <- 1 to 10) { sc.makeRDD(1 to 1000, 1000).count() } Thread.sleep(2000) val l = sc.getClass.getMethod("jobProgressListener").invoke(sc).asInstanceOf[org.apache.spark.ui.jobs.JobProgressListener] org.apache.spark.util.SizeEstimator.estimate(l.stageIdToData) ``` This PR reduces the size of `stageIdToData` from 3487280 to 3009744 (86.3%) in the above case. Author: Shixiong Zhu <shixiong@databricks.com> Closes #18177 from zsxwing/SPARK-20955.
1 parent e11d90b commit 16186cd

File tree

1 file changed

+12
-2
lines changed

1 file changed

+12
-2
lines changed

core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@ package org.apache.spark.ui.jobs
2020
import scala.collection.mutable
2121
import scala.collection.mutable.{HashMap, LinkedHashMap}
2222

23+
import com.google.common.collect.Interners
24+
2325
import org.apache.spark.JobExecutionStatus
2426
import org.apache.spark.executor._
2527
import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo}
@@ -141,6 +143,14 @@ private[spark] object UIData {
141143
}
142144

143145
object TaskUIData {
146+
147+
private val stringInterner = Interners.newWeakInterner[String]()
148+
149+
/** String interning to reduce the memory usage. */
150+
private def weakIntern(s: String): String = {
151+
stringInterner.intern(s)
152+
}
153+
144154
def apply(taskInfo: TaskInfo): TaskUIData = {
145155
new TaskUIData(dropInternalAndSQLAccumulables(taskInfo))
146156
}
@@ -155,8 +165,8 @@ private[spark] object UIData {
155165
index = taskInfo.index,
156166
attemptNumber = taskInfo.attemptNumber,
157167
launchTime = taskInfo.launchTime,
158-
executorId = taskInfo.executorId,
159-
host = taskInfo.host,
168+
executorId = weakIntern(taskInfo.executorId),
169+
host = weakIntern(taskInfo.host),
160170
taskLocality = taskInfo.taskLocality,
161171
speculative = taskInfo.speculative
162172
)

0 commit comments

Comments
 (0)