-
Notifications
You must be signed in to change notification settings - Fork 13
/
Copy pathS3MeasureOutputStream.scala
65 lines (55 loc) · 1.58 KB
/
S3MeasureOutputStream.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
package org.apache.spark.shuffle
import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import java.io.{IOException, OutputStream}
class S3MeasureOutputStream(var out: OutputStream, label: String = "") extends OutputStream with Logging {
private var isOpen = true
private var timings: Long = 0
private var bytes: Long = 0
private def checkOpen(): Unit = {
if (!isOpen) {
throw new IOException("The stream is already closed!")
}
}
override def write(b: Int): Unit = synchronized {
checkOpen()
val now = System.nanoTime()
out.write(b)
timings += System.nanoTime() - now
bytes += 1
}
override def write(b: Array[Byte], off: Int, len: Int): Unit = synchronized {
checkOpen()
val now = System.nanoTime()
out.write(b, off, len)
timings += System.nanoTime() - now
bytes += len
}
override def flush(): Unit = synchronized {
checkOpen()
val now = System.nanoTime()
out.flush()
timings += System.nanoTime() - now
}
override def close(): Unit = synchronized {
if (!isOpen) {
return
}
val now = System.nanoTime()
out.flush()
out.close()
timings += System.nanoTime() - now
out = null
isOpen = false
super.close()
val tc = TaskContext.get()
val sId = tc.stageId()
val sAt = tc.stageAttemptNumber()
val t = timings / 1000000
val bw = bytes.toDouble / (t.toDouble / 1000) / (1024 * 1024)
logInfo(
s"Statistics: Stage ${sId}.${sAt} TID ${tc.taskAttemptId()} -- " +
s"Writing ${label} ${bytes} took ${t} ms (${bw} MiB/s)"
)
}
}