Skip to content

Commit 26b312b

Browse files
author
Ilya Ganelin
committed
Debugging tests
1 parent 17146c2 commit 26b312b

File tree

9 files changed

+28
-31
lines changed

9 files changed

+28
-31
lines changed

core/src/main/scala/org/apache/spark/Aggregator.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -61,8 +61,8 @@ case class Aggregator[K, V, C] (
6161
// Update task metrics if context is not null
6262
// TODO: Make context non optional in a future release
6363
Option(context).foreach { c =>
64-
c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled
65-
c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled
64+
c.taskMetrics.incMemoryBytesSpilled(combiners.memoryBytesSpilled)
65+
c.taskMetrics.incDiskBytesSpilled(combiners.diskBytesSpilled)
6666
}
6767
combiners.iterator
6868
}
@@ -95,8 +95,8 @@ case class Aggregator[K, V, C] (
9595
// Update task metrics if context is not null
9696
// TODO: Make context non-optional in a future release
9797
Option(context).foreach { c =>
98-
c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled
99-
c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled
98+
c.taskMetrics.incMemoryBytesSpilled(combiners.memoryBytesSpilled)
99+
c.taskMetrics.incDiskBytesSpilled(combiners.diskBytesSpilled)
100100
}
101101
combiners.iterator
102102
}

core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -125,8 +125,8 @@ private[spark] class PythonRDD(
125125
init, finish))
126126
val memoryBytesSpilled = stream.readLong()
127127
val diskBytesSpilled = stream.readLong()
128-
context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled
129-
context.taskMetrics.diskBytesSpilled += diskBytesSpilled
128+
context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled)
129+
context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled)
130130
read()
131131
case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
132132
// Signals that an exception has been thrown in python

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -370,7 +370,7 @@ private[spark] class Executor(
370370
if (!taskRunner.attemptedTask.isEmpty) {
371371
Option(taskRunner.task).flatMap(_.metrics).foreach { metrics =>
372372
metrics.updateShuffleReadMetrics
373-
metrics.jvmGCTime = curGCTime - taskRunner.startGCTime
373+
metrics.incJvmGCTime(curGCTime - taskRunner.startGCTime)
374374
if (isLocal) {
375375
// JobProgressListener will hold an reference of it during
376376
// onExecutorMetricsUpdate(), then JobProgressListener can not see

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

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -164,10 +164,10 @@ class TaskMetrics extends Serializable {
164164
private[spark] def updateShuffleReadMetrics() = synchronized {
165165
val merged = new ShuffleReadMetrics()
166166
for (depMetrics <- depsShuffleReadMetrics) {
167-
merged.fetchWaitTime += depMetrics.fetchWaitTime
168-
merged.localBlocksFetched += depMetrics.localBlocksFetched
169-
merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched
170-
merged.remoteBytesRead += depMetrics.remoteBytesRead
167+
merged.incFetchWaitTime(depMetrics.fetchWaitTime)
168+
merged.incLocalBlocksFetched(depMetrics.localBlocksFetched)
169+
merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched)
170+
merged.incRemoteBytesRead(depMetrics.remoteBytesRead)
171171
}
172172
_shuffleReadMetrics = Some(merged)
173173
}
@@ -272,10 +272,7 @@ class ShuffleReadMetrics extends Serializable {
272272
/**
273273
* Number of blocks fetched in this shuffle by this task (remote or local)
274274
*/
275-
private var _totalBlocksFetched: Int = remoteBlocksFetched + localBlocksFetched
276-
def totalBlocksFetched = _totalBlocksFetched
277-
def incTotalBlocksFetched(value: Int) = _totalBlocksFetched += value
278-
def decTotalBlocksFetched(value: Int) = _totalBlocksFetched -= value
275+
def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched
279276
}
280277

281278
/**

core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -159,8 +159,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
159159
for ((it, depNum) <- rddIterators) {
160160
map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum))))
161161
}
162-
context.taskMetrics.memoryBytesSpilled += map.memoryBytesSpilled
163-
context.taskMetrics.diskBytesSpilled += map.diskBytesSpilled
162+
context.taskMetrics.incMemoryBytesSpilled(map.memoryBytesSpilled)
163+
context.taskMetrics.incDiskBytesSpilled(map.diskBytesSpilled)
164164
new InterruptibleIterator(context,
165165
map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]])
166166
}

core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,8 +59,8 @@ private[spark] class HashShuffleReader[K, C](
5959
// the ExternalSorter won't spill to disk.
6060
val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser))
6161
sorter.insertAll(aggregatedIter)
62-
context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled
63-
context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled
62+
context.taskMetrics.incMemoryBytesSpilled(sorter.memoryBytesSpilled)
63+
context.taskMetrics.incDiskBytesSpilled(sorter.diskBytesSpilled)
6464
sorter.iterator
6565
case None =>
6666
aggregatedIter

core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -160,14 +160,14 @@ private[spark] class DiskBlockObjectWriter(
160160
}
161161
finalPosition = file.length()
162162
// In certain compression codecs, more bytes are written after close() is called
163-
writeMetrics.shuffleBytesWritten += (finalPosition - reportedPosition)
163+
writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition)
164164
}
165165

166166
// Discard current writes. We do this by flushing the outstanding writes and then
167167
// truncating the file to its initial position.
168168
override def revertPartialWritesAndClose() {
169169
try {
170-
writeMetrics.shuffleBytesWritten -= (reportedPosition - initialPosition)
170+
writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition)
171171

172172
if (initialized) {
173173
objOut.flush()
@@ -212,14 +212,14 @@ private[spark] class DiskBlockObjectWriter(
212212
*/
213213
private def updateBytesWritten() {
214214
val pos = channel.position()
215-
writeMetrics.shuffleBytesWritten += (pos - reportedPosition)
215+
writeMetrics.incShuffleBytesWritten(pos - reportedPosition)
216216
reportedPosition = pos
217217
}
218218

219219
private def callWithTiming(f: => Unit) = {
220220
val start = System.nanoTime()
221221
f
222-
writeMetrics.shuffleWriteTime += (System.nanoTime() - start)
222+
writeMetrics.incShuffleWriteTime(System.nanoTime() - start)
223223
}
224224

225225
// For testing

core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -156,8 +156,8 @@ final class ShuffleBlockFetcherIterator(
156156
// This needs to be released after use.
157157
buf.retain()
158158
results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf))
159-
shuffleMetrics.remoteBytesRead += buf.size
160-
shuffleMetrics.remoteBlocksFetched += 1
159+
shuffleMetrics.incRemoteBytesRead(buf.size)
160+
shuffleMetrics.incRemoteBlocksFetched(1)
161161
}
162162
logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
163163
}
@@ -233,7 +233,7 @@ final class ShuffleBlockFetcherIterator(
233233
val blockId = iter.next()
234234
try {
235235
val buf = blockManager.getBlockData(blockId)
236-
shuffleMetrics.localBlocksFetched += 1
236+
shuffleMetrics.incLocalBlocksFetched(1)
237237
buf.retain()
238238
results.put(new SuccessFetchResult(blockId, 0, buf))
239239
} catch {
@@ -277,7 +277,7 @@ final class ShuffleBlockFetcherIterator(
277277
currentResult = results.take()
278278
val result = currentResult
279279
val stopFetchWait = System.currentTimeMillis()
280-
shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait)
280+
shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait)
281281

282282
result match {
283283
case SuccessFetchResult(_, size, _) => bytesInFlight -= size

core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -757,12 +757,12 @@ private[spark] class ExternalSorter[K, V, C](
757757
}
758758
}
759759

760-
context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled
761-
context.taskMetrics.diskBytesSpilled += diskBytesSpilled
760+
context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled)
761+
context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled)
762762
context.taskMetrics.shuffleWriteMetrics.filter(_ => bypassMergeSort).foreach { m =>
763763
if (curWriteMetrics != null) {
764-
m.shuffleBytesWritten += curWriteMetrics.shuffleBytesWritten
765-
m.shuffleWriteTime += curWriteMetrics.shuffleWriteTime
764+
m.incShuffleBytesWritten(curWriteMetrics.shuffleBytesWritten)
765+
m.incShuffleWriteTime(curWriteMetrics.shuffleWriteTime)
766766
}
767767
}
768768

0 commit comments

Comments
 (0)