Skip to content

Commit 5525c20

Browse files
author
Ilya Ganelin
committed
Completed refactoring to make vars in TaskMetrics class private
1 parent c64da4f commit 5525c20

File tree

7 files changed

+62
-37
lines changed

7 files changed

+62
-37
lines changed

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -206,10 +206,10 @@ private[spark] class Executor(
206206
val afterSerialization = System.currentTimeMillis()
207207

208208
for (m <- task.metrics) {
209-
m.executorDeserializeTime = taskStart - deserializeStartTime
210-
m.executorRunTime = taskFinish - taskStart
211-
m.jvmGCTime = gcTime - startGCTime
212-
m.resultSerializationTime = afterSerialization - beforeSerialization
209+
m.incExecutorDeserializeTime(taskStart - deserializeStartTime)
210+
m.incExecutorRunTime(taskFinish - taskStart)
211+
m.incJvmGCTime(gcTime - startGCTime)
212+
m.incResultSerializationTime(afterSerialization - beforeSerialization)
213213
}
214214

215215
val accumUpdates = Accumulators.values
@@ -260,8 +260,8 @@ private[spark] class Executor(
260260
val serviceTime = System.currentTimeMillis() - taskStart
261261
val metrics = attemptedTask.flatMap(t => t.metrics)
262262
for (m <- metrics) {
263-
m.executorRunTime = serviceTime
264-
m.jvmGCTime = gcTime - startGCTime
263+
m.incExecutorRunTime(serviceTime)
264+
m.incJvmGCTime(gcTime - startGCTime)
265265
}
266266
val reason = new ExceptionFailure(t, metrics)
267267
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))

core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala

Lines changed: 36 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -39,42 +39,67 @@ class TaskMetrics extends Serializable {
3939
/**
4040
* Host's name the task runs on
4141
*/
42-
var hostname: String = _
43-
42+
private var _hostname: String = _
43+
def hostname = _hostname
44+
def setHostname(value : String) = _hostname = value
45+
4446
/**
4547
* Time taken on the executor to deserialize this task
4648
*/
47-
var executorDeserializeTime: Long = _
48-
49+
private var _executorDeserializeTime: Long = _
50+
def executorDeserializeTime = _executorDeserializeTime
51+
def incExecutorDeserializeTime(value: Long) = _executorDeserializeTime += value
52+
def decExecutorDeserializeTime(value: Long) = _executorDeserializeTime -= value
53+
54+
4955
/**
5056
* Time the executor spends actually running the task (including fetching shuffle data)
5157
*/
52-
var executorRunTime: Long = _
53-
58+
private var _executorRunTime: Long = _
59+
def executorRunTime = _executorRunTime
60+
def incExecutorRunTime(value: Long) = _executorRunTime += value
61+
def decExecutorRunTime(value: Long) = _executorRunTime -= value
62+
5463
/**
5564
* The number of bytes this task transmitted back to the driver as the TaskResult
5665
*/
57-
var resultSize: Long = _
66+
private var _resultSize: Long = _
67+
def resultSize = _resultSize
68+
def incResultSize(value: Long) = _resultSize += value
69+
def decResultSize(value: Long) = _resultSize -= value
70+
5871

5972
/**
6073
* Amount of time the JVM spent in garbage collection while executing this task
6174
*/
62-
var jvmGCTime: Long = _
75+
private var _jvmGCTime: Long = _
76+
def jvmGCTime = _jvmGCTime
77+
def incJvmGCTime(value: Long) = _jvmGCTime += value
78+
def decJvmGCTime(value: Long) = _jvmGCTime -= value
6379

6480
/**
6581
* Amount of time spent serializing the task result
6682
*/
67-
var resultSerializationTime: Long = _
83+
private var _resultSerializationTime: Long = _
84+
def resultSerializationTime = _resultSerializationTime
85+
def incResultSerializationTime(value: Long) = _resultSerializationTime += value
86+
def decResultSerializationTime(value: Long) = _resultSerializationTime -= value
6887

6988
/**
7089
* The number of in-memory bytes spilled by this task
7190
*/
72-
var memoryBytesSpilled: Long = _
91+
private var _memoryBytesSpilled: Long = _
92+
def memoryBytesSpilled = _memoryBytesSpilled
93+
def incMemoryBytesSpilled(value: Long) = _memoryBytesSpilled += value
94+
def decMemoryBytesSpilled(value: Long) = _memoryBytesSpilled -= value
7395

7496
/**
7597
* The number of on-disk bytes spilled by this task
7698
*/
77-
var diskBytesSpilled: Long = _
99+
private var _diskBytesSpilled: Long = _
100+
def diskBytesSpilled = _diskBytesSpilled
101+
def incDiskBytesSpilled(value: Long) = _diskBytesSpilled += value
102+
def decDiskBytesSpilled(value: Long) = _diskBytesSpilled -= value
78103

79104
/**
80105
* If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read

core/src/main/scala/org/apache/spark/scheduler/Task.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex
4747
final def run(attemptId: Long): T = {
4848
context = new TaskContextImpl(stageId, partitionId, attemptId, false)
4949
TaskContextHelper.setTaskContext(context)
50-
context.taskMetrics.hostname = Utils.localHostName()
50+
context.taskMetrics.setHostname(Utils.localHostName())
5151
taskThread = Thread.currentThread()
5252
if (_killed) {
5353
kill(interruptThread = false)

core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul
7676
(deserializedResult, size)
7777
}
7878

79-
result.metrics.resultSize = size
79+
result.metrics.incResultSize(size)
8080
scheduler.handleSuccessfulTask(taskSetManager, tid, result)
8181
} catch {
8282
case cnf: ClassNotFoundException =>

core/src/main/scala/org/apache/spark/util/JsonProtocol.scala

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -592,14 +592,14 @@ private[spark] object JsonProtocol {
592592
return TaskMetrics.empty
593593
}
594594
val metrics = new TaskMetrics
595-
metrics.hostname = (json \ "Host Name").extract[String]
596-
metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long]
597-
metrics.executorRunTime = (json \ "Executor Run Time").extract[Long]
598-
metrics.resultSize = (json \ "Result Size").extract[Long]
599-
metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long]
600-
metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long]
601-
metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long]
602-
metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long]
595+
metrics.setHostname((json \ "Host Name").extract[String])
596+
metrics.incExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long])
597+
metrics.incExecutorRunTime((json \ "Executor Run Time").extract[Long])
598+
metrics.incResultSize((json \ "Result Size").extract[Long])
599+
metrics.incJvmGCTime((json \ "JVM GC Time").extract[Long])
600+
metrics.incResultSerializationTime((json \ "Result Serialization Time").extract[Long])
601+
metrics.incMemoryBytesSpilled((json \ "Memory Bytes Spilled").extract[Long])
602+
metrics.incDiskBytesSpilled((json \ "Disk Bytes Spilled").extract[Long])
603603
metrics.setShuffleReadMetrics(
604604
Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson))
605605
metrics.shuffleWriteMetrics =

core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -227,9 +227,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
227227
shuffleReadMetrics.incRemoteBytesRead(base + 1)
228228
shuffleReadMetrics.incRemoteBlocksFetched(base + 2)
229229
shuffleWriteMetrics.incShuffleBytesWritten(base + 3)
230-
taskMetrics.executorRunTime = base + 4
231-
taskMetrics.diskBytesSpilled = base + 5
232-
taskMetrics.memoryBytesSpilled = base + 6
230+
taskMetrics.incExecutorRunTime(base + 4)
231+
taskMetrics.incDiskBytesSpilled(base + 5)
232+
taskMetrics.incMemoryBytesSpilled(base + 6)
233233
val inputMetrics = new InputMetrics(DataReadMethod.Hadoop)
234234
taskMetrics.inputMetrics = Some(inputMetrics)
235235
inputMetrics.incBytesRead(base + 7)

core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -623,13 +623,13 @@ class JsonProtocolSuite extends FunSuite {
623623
hasHadoopInput: Boolean,
624624
hasOutput: Boolean) = {
625625
val t = new TaskMetrics
626-
t.hostname = "localhost"
627-
t.executorDeserializeTime = a
628-
t.executorRunTime = b
629-
t.resultSize = c
630-
t.jvmGCTime = d
631-
t.resultSerializationTime = a + b
632-
t.memoryBytesSpilled = a + c
626+
t.setHostname("localhost")
627+
t.incExecutorDeserializeTime(a)
628+
t.incExecutorRunTime(b)
629+
t.incResultSize(c)
630+
t.incJvmGCTime(d)
631+
t.incResultSerializationTime(a + b)
632+
t.incMemoryBytesSpilled(a + c)
633633

634634
if (hasHadoopInput) {
635635
val inputMetrics = new InputMetrics(DataReadMethod.Hadoop)

0 commit comments

Comments
 (0)