├── .gitignore
├── core
├── pom.xml
└── src
│ ├── test
│ └── scala
│ │ └── com
│ │ └── cloudera
│ │ └── spark
│ │ └── PeakReportingSuite.scala
│ └── main
│ └── scala
│ ├── com
│ └── cloudera
│ │ └── spark
│ │ ├── PeakReporting.scala
│ │ ├── MemExample.scala
│ │ ├── MemorySnapshot.scala
│ │ ├── MemoryGetter.scala
│ │ ├── Reflector.scala
│ │ ├── SparkNettyMemoryHandle.scala
│ │ └── MemoryMonitor.scala
│ └── org
│ └── apache
│ └── spark
│ ├── executor
│ └── ExecutorPlugin.scala
│ └── memory
│ └── SparkMemoryManagerHandle.scala
├── project
├── build.properties
├── plugins.sbt
└── Build.scala
├── pom.xml
└── README.md
/.gitignore:
--------------------------------------------------------------------------------
1 | .idea
2 | lib_managed
3 | *.iml
4 | **/target
5 | target
6 |
--------------------------------------------------------------------------------
/core/pom.xml:
--------------------------------------------------------------------------------
1 |
5 | 4.0.0
6 |
7 |
8 | com.cloudera
9 | spark-memory-parent_2.11
10 | 0.1.0-SNAPSHOT
11 | ../pom.xml
12 |
13 |
14 | com.cloudera
15 | spark-memory-core_2.11
16 | 0.1.0-SNAPSHOT
17 |
18 |
19 | target/scala-${scala.version}/classes
20 | target/scala-${scala.version}/test-classes
21 |
22 |
23 |
24 |
--------------------------------------------------------------------------------
/project/build.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Licensed to the Apache Software Foundation (ASF) under one or more
3 | # contributor license agreements. See the NOTICE file distributed with
4 | # this work for additional information regarding copyright ownership.
5 | # The ASF licenses this file to You under the Apache License, Version 2.0
6 | # (the "License"); you may not use this file except in compliance with
7 | # the License. You may obtain a copy of the License at
8 | #
9 | # http://www.apache.org/licenses/LICENSE-2.0
10 | #
11 | # Unless required by applicable law or agreed to in writing, software
12 | # distributed under the License is distributed on an "AS IS" BASIS,
13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | # See the License for the specific language governing permissions and
15 | # limitations under the License.
16 | #
17 | sbt.version=0.13.7
18 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/cloudera/spark/PeakReportingSuite.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import org.scalatest.FunSuite
5 |
6 | class PeakReportingSuite extends FunSuite {
7 |
8 | test("increment bytes") {
9 | // delta over 1e7, and 5% increase
10 | assert(IncrementBytes.report(1e9.toLong, 1.051e9.toLong))
11 | // delta over 1e7, but less than 5% increase
12 | assert(!IncrementBytes.report(1e9.toLong, 1.049e9.toLong))
13 |
14 | //5% increase, but below overall threshold
15 | assert(!IncrementBytes.report(1e7.toLong, 1.05e7.toLong))
16 | assert(!IncrementBytes.report(1e7.toLong, 1.9e7.toLong))
17 | assert(!IncrementBytes.report(1e6.toLong, 1e7.toLong))
18 |
19 | // increase from small starting point OK
20 | assert(IncrementBytes.report(0, 1.001e7.toLong))
21 |
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/PeakReporting.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | sealed trait PeakReporting {
5 | def report(orig: Long, update: Long): Boolean
6 | }
7 | case object Never extends PeakReporting {
8 | override def report(orig: Long, update: Long): Boolean = false
9 | }
10 | case object Always extends PeakReporting {
11 | override def report(orig: Long, update: Long): Boolean = update > orig
12 | }
13 | case object IncrementBytes extends PeakReporting {
14 | override def report(orig: Long, update: Long): Boolean = {
15 | val delta = update - orig
16 | delta > 1e7.toInt && (update.toDouble / orig) > 1.05
17 | }
18 | }
19 |
20 | case object IncrementCounts extends PeakReporting {
21 | override def report(orig: Long, update: Long): Boolean = {
22 | val delta = update - orig
23 | delta > 100 && (update.toDouble / orig) > 1.05
24 | }
25 | }
26 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/MemExample.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import java.lang.management.ManagementFactory
5 | import java.nio.ByteBuffer
6 |
7 | import scala.collection.mutable.ArrayBuffer
8 |
9 | object MemExample {
10 |
11 | def main(args: Array[String]): Unit = {
12 | println(ManagementFactory.getRuntimeMXBean.getSpecVersion)
13 | println(ManagementFactory.getRuntimeMXBean.getVmVersion)
14 | MemoryMonitor.listAllMBeans
15 | MemoryMonitor.showLimits
16 | val args = new MemoryMonitorArgs
17 | val monitor = new MemoryMonitor(args)
18 | monitor.installShutdownHook
19 | monitor.beanInfo()
20 | monitor.showMetricNames
21 | monitor.showCurrentMemUsage
22 |
23 | val buffers = ArrayBuffer[ByteBuffer]()
24 |
25 | while (true) {
26 | buffers += ByteBuffer.allocateDirect(1e8.toInt)
27 | monitor.updateAndMaybeShowPeaks()
28 | }
29 | }
30 | }
31 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | scalaVersion := "2.10.4"
2 |
3 | resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns)
4 |
5 | resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/"
6 |
7 | resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/"
8 |
9 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2")
10 |
11 | addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0")
12 |
13 | addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0")
14 |
15 | addSbtPlugin("com.typesafe.sbt" % "sbt-pom-reader" % "2.0.0")
16 |
17 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4")
18 |
19 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.7.0")
20 |
21 | addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1")
22 |
23 | addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3")
24 |
25 | addSbtPlugin("io.spray" % "sbt-revolver" % "0.7.2")
26 |
27 | libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3"
28 |
29 | libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3"
30 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/MemorySnapshot.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | class MemorySnapshot(
5 | val time: Long,
6 | val values: Array[Long]
7 | )
8 |
9 | class MemoryPeaks(
10 | val values: Array[Long],
11 | val peakTimes: Array[Long]) {
12 | def this(n:Int) {
13 | this(new Array[Long](n), new Array[Long](n))
14 | }
15 |
16 | def update(
17 | snapshot: MemorySnapshot,
18 | updates: PeakUpdate,
19 | reporting: IndexedSeq[PeakReporting]): Boolean = {
20 | assert(snapshot.values.length == values.length)
21 | var nUpdates = 0
22 | (0 until values.length).foreach { idx =>
23 | val orig = values(idx)
24 | val update = snapshot.values(idx)
25 | if (reporting(idx).report(orig, update)) {
26 | values(idx) = snapshot.values(idx)
27 | peakTimes(idx) = snapshot.time
28 | updates.updateIdx(nUpdates) = idx
29 | updates.delta(nUpdates) = update - orig
30 | nUpdates += 1
31 | }
32 | }
33 | updates.nUpdates = nUpdates
34 | nUpdates != 0
35 | }
36 | }
37 |
38 | class PeakUpdate(
39 | val updateIdx: Array[Int],
40 | val delta: Array[Long],
41 | var nUpdates: Int
42 | ) {
43 | def this(n: Int) {
44 | this(new Array[Int](n), new Array[Long](n), 0)
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/executor/ExecutorPlugin.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.spark.executor
18 |
19 | import org.apache.spark.TaskContext
20 | import org.apache.spark.util.{TaskFailureListener, TaskCompletionListener}
21 |
22 | /**
23 | * This really needs to be in the spark source. But because I don't have a release artifact to
24 | * build against with this, I
25 | */
26 | trait ExecutorPlugin extends TaskCompletionListener with TaskFailureListener {
27 | def taskStart(taskContext: TaskContext): Unit
28 | }
29 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/MemoryGetter.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import java.lang.management.{BufferPoolMXBean, MemoryMXBean, MemoryPoolMXBean}
5 |
6 | trait MemoryGetter {
7 | def namesAndReporting: Seq[(String, PeakReporting)]
8 | def values(dest: Array[Long], offset: Int): Unit
9 | }
10 |
11 | class MemoryMxBeanGetter(bean: MemoryMXBean) extends MemoryGetter {
12 | val namesAndReporting: Seq[(String, PeakReporting)] = for {
13 | source <- Seq("heap", "offheap")
14 | usage <- Seq(("used", IncrementBytes), ("committed", Always))
15 | } yield {
16 | (source + ":" + usage._1, usage._2)
17 | }
18 | def values(dest: Array[Long], offset:Int): Unit = {
19 | val heap = bean.getHeapMemoryUsage()
20 | dest(offset) = heap.getUsed()
21 | dest(offset + 1) = heap.getCommitted()
22 | val offheap = bean.getNonHeapMemoryUsage()
23 | dest(offset + 2) = offheap.getUsed()
24 | dest(offset + 3) = offheap.getCommitted()
25 | }
26 | }
27 |
28 | class PoolGetter(bean: MemoryPoolMXBean) extends MemoryGetter {
29 | val namesAndReporting: Seq[(String, PeakReporting)] =
30 | Seq(("used", IncrementBytes), ("committed", Always)).map { case (n, r) =>
31 | (bean.getName() + n, r)
32 | }
33 | def values(dest: Array[Long], offset: Int): Unit = {
34 | // there are actually a bunch more things here I *could* get ...
35 | val usage = bean.getUsage()
36 | dest(offset) = usage.getUsed()
37 | dest(offset + 1) = usage.getCommitted()
38 | }
39 | }
40 |
41 | class BufferPoolGetter(bean: BufferPoolMXBean) extends MemoryGetter {
42 | val namesAndReporting = Seq(("capacity", IncrementBytes), ("used", IncrementBytes)).map{ case (n, r) =>
43 | (bean.getName() + ":" + n, r)
44 | }
45 | def values(dest: Array[Long], offset: Int): Unit = {
46 | dest(offset) = bean.getTotalCapacity()
47 | dest(offset + 1) = bean.getMemoryUsed()
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/memory/SparkMemoryManagerHandle.scala:
--------------------------------------------------------------------------------
1 | package org.apache.spark.memory
2 |
3 | import com.cloudera.spark.{Reflector, IncrementBytes, MemoryGetter}
4 | import org.apache.spark.util.{Utils, ThreadStackTrace}
5 | import org.apache.spark.{SparkContext, SparkEnv}
6 |
7 | class SparkMemoryManagerHandle(
8 | val onHeapStorage: StorageMemoryPool,
9 | val offHeapStorage: StorageMemoryPool,
10 | val onHeapExecution: ExecutionMemoryPool,
11 | val offHeapExecution: ExecutionMemoryPool,
12 | val lock: Object) extends MemoryGetter {
13 |
14 | val namesAndReporting = for {
15 | t <- Seq("onHeap", "offHeap")
16 | s <- Seq("Storage", "Execution", "MaxTaskExecution")
17 | } yield {
18 | (s"$t$s", IncrementBytes)
19 | }
20 |
21 | override def values(dest: Array[Long], offset: Int): Unit = {
22 | dest(offset) = onHeapStorage.memoryUsed
23 | dest(offset + 1) = onHeapExecution.memoryUsed
24 | dest(offset + 2) = maxTaskExecution(onHeapExecution)
25 | dest(offset + 3) = offHeapStorage.memoryUsed
26 | dest(offset + 4) = offHeapExecution.memoryUsed
27 | dest(offset + 5) = maxTaskExecution(offHeapExecution)
28 | }
29 |
30 | def maxTaskExecution(executionMemoryPool: ExecutionMemoryPool): Long = {
31 | // This is totally hacking into internals, even locks ...
32 | import Reflector._
33 | lock.synchronized {
34 | val taskMem = executionMemoryPool
35 | .reflectField("memoryForTask").asInstanceOf[scala.collection.Map[Long, Long]]
36 | if (taskMem.nonEmpty) {
37 | taskMem.values.max
38 | } else {
39 | 0L
40 | }
41 | }
42 | }
43 | }
44 |
45 | object SparkMemoryManagerHandle {
46 | def get(displayError: Boolean = false): Option[SparkMemoryManagerHandle] = try {
47 | val env = SparkEnv.get
48 | val memManager = env.memoryManager
49 | import Reflector._
50 | Some(new SparkMemoryManagerHandle(
51 | memManager.reflectField("onHeapStorageMemoryPool").asInstanceOf[StorageMemoryPool],
52 | memManager.reflectField("offHeapStorageMemoryPool").asInstanceOf[StorageMemoryPool],
53 | memManager.reflectField("onHeapExecutionMemoryPool").asInstanceOf[ExecutionMemoryPool],
54 | memManager.reflectField("offHeapExecutionMemoryPool").asInstanceOf[ExecutionMemoryPool],
55 | memManager
56 | ))
57 | } catch {
58 | case ex: Exception =>
59 | if (displayError) {
60 | ex.printStackTrace()
61 | }
62 | None
63 | }
64 |
65 | def isDynamicAllocation(sc: SparkContext): Boolean = {
66 | org.apache.spark.util.Utils.isDynamicAllocationEnabled(sc.getConf)
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/Reflector.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import java.lang.reflect.{Field, Method}
5 |
6 | /* For example, I want to do this:
7 | *
8 | * sqlContext.catalog.client.getTable("default", "blah").properties
9 | *
10 | * but none of that is public to me in the shell. Using this, I can now do:
11 | *
12 | * sqlContext.reflectField("catalog").reflectField("client").reflectMethod("getTable", Seq("default", "blah")).reflectField("properties")
13 | *
14 | * not perfect, but usable.
15 | */
16 | object Reflector {
17 |
18 | def methods(obj: Any): Seq[String] = {
19 | _methods(obj.getClass()).map(_.getName()).sorted
20 | }
21 |
22 | def _methods(cls: Class[_]): Seq[Method] = {
23 | if (cls == null) Seq()
24 | else cls.getDeclaredMethods() ++ _methods(cls.getSuperclass())
25 | }
26 |
27 | def fields(obj: Any): Seq[String] = {
28 | _fields(obj.getClass()).map(_.getName()).sorted
29 | }
30 |
31 | def _fields(cls: Class[_]): Seq[Field] = {
32 | if (cls == null) Seq()
33 | else cls.getDeclaredFields() ++ _fields(cls.getSuperclass())
34 | }
35 |
36 | def findMethod(obj: Any, name: String): Method = {
37 | // TODO: handle scala's name munging, eg. org$apache$spark$sql$hive$HiveExternalCatalog$$makeQualified
38 | val method = _methods(obj.getClass()).find(_.getName() == name).get
39 | method.setAccessible(true)
40 | method
41 | }
42 |
43 | def get(obj: Any, name: String): Any = {
44 | val clz = obj.getClass()
45 | val fields = _fields(clz)
46 | fields.find(_.getName() == name).orElse {
47 | // didn't find an exact match, try again with name munging that happens for private vars
48 | fields.find(_.getName().endsWith("$$" + name))
49 | } match {
50 | case Some(f) =>
51 | f.setAccessible(true)
52 | f.get(obj)
53 | case None =>
54 | // not a field, maybe its actually a method in byte code
55 | val m = findMethod(obj, name)
56 | m.invoke(obj)
57 | }
58 | }
59 |
60 | def showFields(obj: Any): Unit = {
61 | fields(obj).foreach {
62 | println
63 | }
64 | }
65 |
66 | def showMethods(obj: Any): Unit = {
67 | methods(obj).foreach {
68 | println
69 | }
70 | }
71 |
72 | implicit class ReflectorConversions(obj: Any) {
73 | // TODO error msgs
74 | def reflectField(name: String): Any = {
75 | get(obj, name)
76 | }
77 |
78 | def reflectMethod(name: String, args: Seq[Object]): Any = {
79 | // TODO find a method that actually matches the args, not just the name, to deal w/ overloading
80 | findMethod(obj, name).invoke(obj, args: _*)
81 | }
82 | }
83 |
84 | }
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
5 | 4.0.0
6 |
7 |
8 | com.cloudera
9 | spark-memory-parent_2.11
10 | 0.1.0-SNAPSHOT
11 | pom
12 |
13 |
14 |
15 | cloudera
16 | https://repository.cloudera.com/artifactory/cloudera-repos/
17 |
18 |
19 |
20 |
21 | core
22 |
23 |
24 |
25 | 2.11.8
26 | 1.8
27 | 64m
28 | 512m
29 | 512m
30 |
31 |
32 |
33 |
34 |
35 | com.quantifind
36 | sumac_2.11
37 | 0.3.0
38 |
39 |
40 |
41 | org.scala-lang
42 | scala-library
43 | ${scala.version}
44 | provided
45 |
46 |
47 | org.apache.spark
48 | spark-core_2.11
49 | 2.3.0
50 | provided
51 |
52 |
53 |
54 | org.scalatest
55 | scalatest_2.11
56 | 3.0.5
57 | test
58 |
59 |
60 |
61 |
62 |
63 |
64 |
65 |
66 | net.alchim31.maven
67 | scala-maven-plugin
68 | 3.2.0
69 |
70 | ${scala.version}
71 | incremental
72 | true
73 |
74 | -unchecked
75 | -deprecation
76 | -feature
77 |
78 |
79 | -Xms1024m
80 | -Xmx1024m
81 | -XX:PermSize=${PermGen}
82 | -XX:MaxPermSize=${MaxPermGen}
83 | -XX:ReservedCodeCacheSize=${CodeCacheSize}
84 |
85 |
86 | -source
87 | ${java.version}
88 | -target
89 | ${java.version}
90 |
91 |
92 |
93 |
94 | scala-compile-first
95 | process-resources
96 |
97 | compile
98 |
99 |
100 |
101 | scala-test-compile-first
102 | process-test-resources
103 |
104 | testCompile
105 |
106 |
107 |
108 |
109 |
110 | org.apache.maven.plugins
111 | maven-compiler-plugin
112 | 3.3
113 |
114 | ${java.version}
115 | ${java.version}
116 | UTF-8
117 | 1024m
118 | true
119 |
120 |
121 |
122 |
123 |
124 |
125 | net.alchim31.maven
126 | scala-maven-plugin
127 |
128 |
129 | maven-assembly-plugin
130 |
131 |
132 | jar-with-dependencies
133 |
134 |
135 |
136 |
137 | make-assembly
138 | package
139 |
140 | single
141 |
142 |
143 |
144 |
145 |
146 |
147 |
148 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | Spark Memory Monitor
2 | ========================
3 |
4 | Usage
5 | -----------
6 |
7 | Build with `mvn package`, `sbt`, etc.
8 |
9 | Include that jar in your spark application. You could bundle it directly, or just include it with `--jars`.
10 |
11 | The monitoring is configured via java system properties:
12 |
13 | * "memory.monitor.enabled=true" -- if its set, enable monitoring (regardless of value)
14 | * "memory.monitor.freq=[millis]" -- set the frequency of polling used to detect peaks, in millis
15 |
16 | So a typical invocation might look like:
17 |
18 | ```
19 | spark-submit \
20 | ...
21 | --jars spark-memory-core_2.11-0.1.0-SNAPSHOT-jar-with-dependencies.jar \
22 | --driver-java-options "-XX:MaxMetaspaceSize=200M -XX:+PrintGCDetails -Dmemory.monitor.enabled=true -Dmemory.monitor.freq=100" \
23 | --conf spark.executor.extraJavaOptions="-XX:MaxMetaspaceSize=200M -XX:+PrintGCDetails -Dmemory.monitor.enabled=true -Dmemory.monitor.freq=100" \
24 | --conf spark.executor.plugins="com.cloudera.spark.MemoryMonitorExecutorExtension" \
25 | ...
26 | ```
27 |
28 | This includes the Dynamic Allocation "plugin" (see below) -- note that requires a patched spark and the api may change.
29 |
30 | You *also* have to modify the code of your job itself to turn it on. Below are code samples (using reflection so your
31 | build doesn't need to know about the memory monitor at all).
32 |
33 |
34 | Installing on Driver
35 | -----------------------
36 |
37 | ```scala
38 | sys.props.get("memory.monitor.enabled").foreach { _ =>
39 | val clz = Class.forName("com.cloudera.spark.MemoryMonitor")
40 | val m = clz.getMethod("installIfSysProps")
41 | m.invoke(null)
42 | }
43 | ```
44 |
45 |
46 | Installing on Executors, with Static Allocation
47 | -------------------------------------------------
48 |
49 | Be sure to include this conf:
50 |
51 | ```scala
52 | --conf "spark.scheduler.minRegisteredResourcesRatio=1.0"
53 | ```
54 |
55 | and then in your job:
56 |
57 | ```scala
58 | sys.props.get("memory.monitor.enabled").foreach { _ =>
59 | val clz = Class.forName("com.cloudera.spark.MemoryMonitor")
60 | val m = clz.getMethod("installOnExecIfStaticAllocation", classOf[org.apache.spark.SparkContext])
61 | m.invoke(null, sc)
62 | }
63 | ```
64 |
65 | Installing on Executors, with DynamicAllocation, and "Plugin"
66 | -------------------------------------------------------------
67 |
68 | Its trickier to get the MemoryMonitor installed on the executors with DynamicAllocation. Executors can come and go at any time,
69 | and we don't have a good way to initialize something before the first task is run.
70 |
71 | You can modify spark to expose some sort of "executor-plugin api", like [this](https://github.com/squito/spark/commit/0ca94828e88006d2efeed6d106f4f0495cf3f5ee).
72 | (Hopefully this can be exposed as part of spark eventually, I'll open a jira once I think about the api a little more.)
73 | Then you'd reference the MemoryMonitor as an extension class:
74 |
75 | ```
76 | --conf spark.executor.plugins="com.cloudera.spark.MemoryMonitorExecutorExtension"
77 | ```
78 |
79 | Understanding the Output
80 | =========================
81 |
82 | The MemoryMonitor will poll the memory usage of a variety of subsystems used by Spark. It tracks the memory of the JVM itself,
83 | as well as offheap memory which is untracked by the JVM. In addition it will report all updates to _peak_ memory use of each
84 | subsystem, and log just the peaks. This helps reduce the overhead and also make it manageable to view the logs. Finally, it
85 | registers a shutdown hook, to report both the current usage of each memory metric, and also the peak since it started monitoring.
86 |
87 | The report is currently a grab-bag of many different metrics, but the most important are probably:
88 |
89 | * `offheap:committed` -- in Java8, the memory used for code (aka the metaspace, though the JVM uses that term for
90 | something slightly more specific)
91 | * `netty-[subsystem]-numActiveBytes` -- bytes that netty has allocated in its offheap memory pools
92 | * `netty-[subsystem]-directAllocatedUnused` -- bytes that netty has allocated in its offheap memory pools that are currently *unused*
93 | * `netty-[subsystem]-heapAllocatedUnused` -- bytes that netty has allocated in its heap memory pools that are currently *unused*
94 | * `on/offHeapStorage` -- bytes used by spark's block storage
95 | * `on/offHeapExecution` -- bytes used by spark's execution layer
96 | * `on/offHeapMaxTaskExecution` -- the maximum number of "execution" bytes spark has assigned to any single task
97 |
98 | Eg., will see sections of the stdout which just contain log lines of the form:
99 |
100 | ```
101 | Peak Memory updates:Mon Jun 25 10:51:07 PDT 2018
102 | offheap:committed : 94.0 MB (+448.0 KB)
103 | Code Cachecommitted : 28.4 MB (+192.0 KB)
104 | Metaspacecommitted : 58.0 MB (+256.0 KB)
105 | ```
106 |
107 | which show when the JVM is busy loading classes (perhaps even spark codegen classes)
108 |
109 | Then when there is a burst of messages that have to be handled, you may see messages more like this:
110 |
111 | ```
112 | netty-blockTransfer-server-usedHeapMem : 16.0 MB (+16.0 MB)
113 | netty-blockTransfer-server-usedDirectMem : 16.0 MB (+16.0 MB)
114 | netty-blockTransfer-server-numThreadLocalCaches : 8.0 B (+4.0 B)
115 | netty-blockTransfer-server-numActiveBytes : 16.0 MB (+16.0 MB)
116 | netty-blockTransfer-server-directAllocatedUnused : 15.9 MB (+15.9 MB)
117 | netty-blockTransfer-server-heapAllocationUnused : 15.9 MB (+15.9 MB)
118 | ```
119 |
120 | Note that polling is imprecise. The memory usage can spike very quickly. You can increase the polling frequency, though this
121 | may have adverse effects on performance. (Currently we have not evaluated the performance impact **AT ALL**, so you're on your
122 | own).
123 |
124 | You can get around this limitation a *little* bit with the shutdown hook. Hopefully, the memory usage in the shutdown hook might
125 | give you some indication of the final memory usage -- especially when spark is killed by the OS / cluster manager. However,
126 | shutdown hooks are best effort; they may not run, and even when they do, the JVM will be performing cleanup concurrently so you
127 | might be observing memory in a different state.
128 |
129 | In any case, the shut down metrics can be very valuable -- they look something like this:
130 |
131 | ```
132 | IN SHUTDOWN
133 | Mem usage at Mon Jun 25 11:03:51 PDT 2018
134 | ===============
135 | heap:used :1498.4 MB(1571212240)
136 | heap:committed :1902.0 MB(1994391552)
137 | ...
138 |
139 | Peak Memory usage so far Mon Jun 25 11:03:51 PDT 2018
140 | heap:used :1257.2 MB Mon Jun 25 10:53:14 PDT 2018
141 | heap:committed :1999.0 MB Mon Jun 25 10:51:34 PDT 2018
142 | offheap:used :162.4 MB Mon Jun 25 10:51:31 PDT 2018
143 | ...
144 | ```
145 |
146 | TODO
147 | =======
148 |
149 | * understand the performance impact
150 | * more things to monitor -- eg. parquet? is that covered by jvm metrics?
151 | * spark plugin api, so we can use this w/ dynamic allocation
152 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/SparkNettyMemoryHandle.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import java.lang.reflect.Modifier
5 |
6 | import scala.collection.JavaConverters._
7 |
8 | import io.netty.buffer._
9 | import io.netty.channel.ChannelOption
10 |
11 | import org.apache.spark.SparkEnv
12 |
13 | import scala.util.Try
14 |
15 | class SparkNettyMemoryHandle(
16 | val rpcClientPool: PooledByteBufAllocatorMetric,
17 | val rpcServerPool: Option[PooledByteBufAllocatorMetric],
18 | val blockClientPool: PooledByteBufAllocatorMetric,
19 | val blockServerPool: PooledByteBufAllocatorMetric,
20 | val externalShuffleClientPool: Option[PooledByteBufAllocatorMetric]
21 | ) extends MemoryGetter {
22 | override def toString(): String = {
23 | "RPC Client pool:" + rpcClientPool + "\n" +
24 | "RPC Server pool:" + rpcServerPool + "\n" +
25 | "Block Transfer client pool:" + blockClientPool + "\n" +
26 | "Block Transfer server pool:" + blockServerPool + "\n" +
27 | "External Shuffle Client pool:" + externalShuffleClientPool
28 | }
29 |
30 | val poolMetrics = Seq(
31 | ("usedHeapMem", IncrementBytes),
32 | ("usedDirectMem", IncrementBytes),
33 | ("numHeapArenas", Always),
34 | ("numDirectArenas", Always),
35 | ("numThreadLocalCaches", Always))
36 |
37 | val allPooledMetrics = poolMetrics ++ SparkNettyMemoryHandle.VERBOSE_METRICS ++
38 | Seq(("directAllocatedUnused", IncrementBytes), ("heapAllocationUnused", IncrementBytes))
39 |
40 | val poolsAndNames: Seq[(PooledByteBufAllocatorMetric, String)] = Seq(
41 | Some(rpcClientPool, "rpc-client"),
42 | rpcServerPool.map((_, "rpc-server")),
43 | Some(blockClientPool, "blockTransfer-client"),
44 | Some(blockServerPool, "blockTransfer-server"),
45 | externalShuffleClientPool.map((_, "external-shuffle-client"))
46 | ).flatten
47 |
48 | val pools = poolsAndNames.map(_._1)
49 |
50 | override val namesAndReporting: Seq[(String, PeakReporting)] = (for {
51 | (_, poolName) <- poolsAndNames
52 | (metric, reporting) <- allPooledMetrics
53 | } yield {
54 | ("netty-" + poolName + "-" + metric, reporting)
55 | }) ++ Seq(
56 | ("netty-Unpooled-heapUsed", IncrementBytes),
57 | ("netty-Unpooled-directUsed", IncrementBytes)
58 | )
59 |
60 | def values(dest: Array[Long], initOffset: Int): Unit = {
61 | var offset = initOffset
62 | pools.foreach { pool =>
63 | dest(offset) = pool.usedHeapMemory()
64 | dest(offset + 1) = pool.usedDirectMemory()
65 | dest(offset + 2) = pool.numHeapArenas()
66 | dest(offset + 3) = pool.numDirectArenas()
67 | dest(offset + 4) = pool.numThreadLocalCaches()
68 | // TODO more from heap arenas?
69 | SparkNettyMemoryHandle.getArenaMetrics(pool.directArenas().asScala, dest, offset + 5)
70 | dest(offset + allPooledMetrics.length - 1) =
71 | SparkNettyMemoryHandle.poolArenaFreeBytes(pool.heapArenas().asScala)
72 | offset += allPooledMetrics.length
73 | }
74 | // NOTE: netty 4.1 only, so spark 2.3+ only
75 | val unpooledMetric = UnpooledByteBufAllocator.DEFAULT.metric()
76 | dest(offset) = unpooledMetric.usedHeapMemory()
77 | dest(offset + 1) = unpooledMetric.usedDirectMemory()
78 |
79 | }
80 | }
81 |
82 | object SparkNettyMemoryHandle {
83 |
84 | def get(displayError: Boolean = false): Option[SparkNettyMemoryHandle] = {
85 | Option(SparkEnv.get).map { env =>
86 | new SparkNettyMemoryHandle(
87 | getRpcClientPooledAllocator(env).metric,
88 | Try(getRpcServerPooledAllocator(env).metric).toOption,
89 | getBlockTransferServiceClientPooledAllocator(env).metric,
90 | getBlockTransferServiceServerPooledAllocator(env).metric,
91 | getExternalShuffleServiceClientPooledAllocator(env).map(_.metric)
92 | )
93 | }
94 | }
95 |
96 | def getBlockTransferServiceClientPooledAllocator(env: SparkEnv): PooledByteBufAllocator = {
97 | import Reflector._
98 | val ftory = env.blockManager.reflectField("blockTransferService").reflectField("clientFactory")
99 | getPooledAllocatorFromClientFactory(ftory)
100 | }
101 |
102 | def getExternalShuffleServiceClientPooledAllocator(env: SparkEnv): Option[PooledByteBufAllocator] = {
103 | import Reflector._
104 | val shuffleClient = env.blockManager.reflectField("shuffleClient")
105 | println(s"shuffleClient = $shuffleClient (${shuffleClient.getClass()})")
106 | if (shuffleClient.getClass().getSimpleName.endsWith("ExternalShuffleClient")) {
107 | Some(getPooledAllocatorFromClientFactory(shuffleClient.reflectField("clientFactory")))
108 | } else {
109 | None
110 | }
111 | }
112 |
113 | def getBlockTransferServiceServerPooledAllocator(env: SparkEnv): PooledByteBufAllocator = {
114 | import Reflector._
115 | val server = env.blockManager.reflectField("blockTransferService").reflectField("server")
116 | getServerPooledAllocator(server)
117 | }
118 |
119 | def getRpcClientPooledAllocator(env: SparkEnv): PooledByteBufAllocator = {
120 | import Reflector._
121 | val ftory = env.reflectField("rpcEnv").reflectField("clientFactory")
122 | getPooledAllocatorFromClientFactory(ftory)
123 | }
124 |
125 | def getRpcServerPooledAllocator(env: SparkEnv): PooledByteBufAllocator = {
126 | import Reflector._
127 | val server = env.reflectField("rpcEnv").reflectField("server")
128 | getServerPooledAllocator(server)
129 | }
130 |
131 | def getPooledAllocatorFromClientFactory(clientFactory: Any): PooledByteBufAllocator = {
132 | assert(clientFactory.getClass().getSimpleName.endsWith("TransportClientFactory"))
133 | import Reflector._
134 | clientFactory.reflectField("pooledAllocator").asInstanceOf[PooledByteBufAllocator]
135 | }
136 |
137 | def getServerPooledAllocator(server: Any): PooledByteBufAllocator = {
138 | assert(server.getClass().getSimpleName.endsWith("TransportServer"))
139 | // this looks like the best way to get it even in old versions (nettyMetric is only in 2.3+)
140 | import Reflector._
141 | val serverOptions = server.reflectField("bootstrap").reflectField("options")
142 | .asInstanceOf[java.util.Map[ChannelOption[_], Object]]
143 | serverOptions.get(ChannelOption.ALLOCATOR).asInstanceOf[PooledByteBufAllocator]
144 | }
145 |
146 | def getArenaMetrics(arenaMetrics: Seq[PoolArenaMetric], dest: Array[Long], offset: Int): Unit = {
147 | SparkNettyMemoryHandle.metricMethods.zipWithIndex.foreach { case (metric, idx) =>
148 | var total = 0L
149 | if (metric.getReturnType() == classOf[Int]){
150 | arenaMetrics.foreach { arena => total += metric.invoke(arena).asInstanceOf[Int] }
151 | } else {
152 | arenaMetrics.foreach { arena => total += metric.invoke(arena).asInstanceOf[Long] }
153 | }
154 | dest(offset + idx) = total
155 | }
156 |
157 | dest(offset + metricMethods.size) = poolArenaFreeBytes(arenaMetrics)
158 |
159 | }
160 |
161 | val VERBOSE_METRICS = Seq(
162 | "numAllocations",
163 | "numTinyAllocations",
164 | "numSmallAllocations",
165 | "numNormalAllocations",
166 | "numHugeAllocations",
167 | "numDeallocations",
168 | "numTinyDeallocations",
169 | "numSmallDeallocations",
170 | "numNormalDeallocations",
171 | "numHugeDeallocations",
172 | "numActiveAllocations",
173 | "numActiveTinyAllocations",
174 | "numActiveSmallAllocations",
175 | "numActiveNormalAllocations",
176 | "numActiveHugeAllocations"
177 | ).map((_, IncrementCounts)) ++ Seq(("numActiveBytes", IncrementBytes))
178 |
179 | val metricMethods = VERBOSE_METRICS.flatMap { case (methodName, _) =>
180 | val m = classOf[PoolArenaMetric].getMethod(methodName)
181 | if (Modifier.isPublic(m.getModifiers())) {
182 | Some(m)
183 | } else {
184 | None
185 | }
186 | }
187 |
188 | def poolArenaFreeBytes(arenas: Seq[PoolArenaMetric]): Long = {
189 | var total = 0L
190 | for {
191 | arena <- arenas
192 | list <- arena.chunkLists.asScala
193 | metric <- list.asScala
194 | } {
195 | total += metric.freeBytes()
196 | }
197 | total
198 | }
199 |
200 | }
201 |
--------------------------------------------------------------------------------
/project/Build.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | import java.io._
19 |
20 | import scala.collection.JavaConversions._
21 | import scala.util.Properties
22 |
23 | import sbt._
24 | import sbt.Classpaths.publishTask
25 | import sbt.Keys._
26 | import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion
27 | import com.typesafe.sbt.pom.{loadEffectivePom, PomBuild, SbtPomKeys}
28 |
29 | import spray.revolver.RevolverPlugin._
30 |
31 | object BuildCommons {
32 |
33 | private val buildLocation = file(".").getAbsoluteFile.getParentFile
34 | val testTempDir = buildLocation + "/target/tmp"
35 |
36 | val allProjects @ Seq(core, examples) =
37 | Seq("core", "examples").map(ProjectRef(buildLocation, _))
38 | val assemblyProjects @ Seq(assembly) =
39 | Seq("assembly").map(ProjectRef(buildLocation, _))
40 |
41 | }
42 |
43 | object MyBuild extends PomBuild {
44 |
45 | import BuildCommons._
46 | import scala.collection.mutable.Map
47 |
48 | val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty
49 |
50 | override val userPropertiesMap = System.getProperties.toMap
51 |
52 | lazy val MavenCompile = config("m2r") extend(Compile)
53 | lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
54 |
55 | lazy val sharedSettings = Seq (
56 | javaHome := sys.env.get("JAVA_HOME")
57 | .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() })
58 | .map(file),
59 | incOptions := incOptions.value.withNameHashing(true),
60 | retrieveManaged := true,
61 | retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]",
62 | publishMavenStyle := true,
63 |
64 | resolvers += Resolver.mavenLocal,
65 | otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))),
66 | publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map {
67 | (arts, _, level) => new PublishConfiguration(None, "dotM2", arts, Seq(), level)
68 | },
69 | publishMavenStyle in MavenCompile := true,
70 | publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal),
71 | publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn,
72 |
73 | javacOptions in (Compile, doc) ++= {
74 | val Array(major, minor, _) = System.getProperty("java.version").split("\\.", 3)
75 | if (major.toInt >= 1 && minor.toInt >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty
76 | },
77 |
78 | javacOptions in Compile ++= Seq("-encoding", "UTF-8", "-source", "1.8", "-target", "1.8"),
79 |
80 | // this lets us run spark apps from within sbt, but still leave it as a "provided" dependency, so its
81 | // not bundled into jars. See http://stackoverflow.com/questions/18838944/how-to-add-provided-dependencies-back-to-run-test-tasks-classpath
82 | runMain in Compile <<= Defaults.runMainTask(fullClasspath in Compile, runner in (Compile, run))
83 | )
84 |
85 | def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = {
86 | val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]())
87 | projectsMap += (projectRef.project -> (existingSettings ++ settings))
88 | }
89 |
90 | // Note ordering of these settings matter.
91 | /* Enable shared settings on all projects */
92 | (allProjects ++ assemblyProjects)
93 | .foreach(enable(sharedSettings ++ Revolver.settings))
94 |
95 | /* Enable tests settings for all projects */
96 | allProjects.foreach(enable(TestSettings.settings))
97 |
98 | /* Enable Assembly for all projects */
99 | enable(Assembly.settings)(core)
100 |
101 | // TODO: move this to its upstream project.
102 | override def projectDefinitions(baseDirectory: File): Seq[Project] = {
103 | super.projectDefinitions(baseDirectory).map { x =>
104 | if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*)
105 | else x.settings(Seq[Setting[_]](): _*)
106 | }
107 | }
108 |
109 | }
110 |
111 | object Assembly {
112 | import sbtassembly.AssemblyUtils._
113 | import sbtassembly.Plugin._
114 | import AssemblyKeys._
115 |
116 | val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.")
117 |
118 | lazy val settings = assemblySettings ++ Seq(
119 | test in assembly := {},
120 | assemblyOption in assembly := (assemblyOption in assembly).value.copy(includeScala = false),
121 | mergeStrategy in assembly := {
122 | case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard
123 | case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
124 | case m if m.toLowerCase.matches("meta-inf.*\\.sf") => MergeStrategy.discard
125 | case "log4j.properties" => MergeStrategy.discard
126 | case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines
127 | case "reference.conf" => MergeStrategy.concat
128 | case _ => MergeStrategy.first
129 | }
130 | )
131 | }
132 |
133 | object TestSettings {
134 | import BuildCommons._
135 |
136 | lazy val settings = Seq (
137 | // Fork new JVMs for tests and set Java options for those
138 | fork := true,
139 | // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes
140 | // launched by the tests have access to the correct test-time classpath.
141 | envVars in Test ++= Map(
142 | "SPARK_DIST_CLASSPATH" ->
143 | (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"),
144 | "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))),
145 | javaOptions in Test += s"-Djava.io.tmpdir=" + testTempDir,
146 | javaOptions in Test += "-Dspark.port.maxRetries=100",
147 | javaOptions in Test += "-Dspark.ui.enabled=false",
148 | javaOptions in Test += "-Dspark.ui.showConsoleProgress=false",
149 | javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true",
150 | javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true",
151 | javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true",
152 | javaOptions in Test += "-Dderby.system.durability=test",
153 | javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark")
154 | .map { case (k,v) => "-D" + k + "=" + v }.toSeq,
155 | javaOptions in Test += "-ea",
156 | javaOptions in Test ++= "-Xmx3g -Xss4096k -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g"
157 | .split(" ").toSeq,
158 | javaOptions += "-Xmx3g",
159 | // Show full stack trace and duration in test cases.
160 | testOptions in Test += Tests.Argument("-oDF"),
161 | testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"),
162 | // Enable Junit testing.
163 | libraryDependencies += "com.novocode" % "junit-interface" % "0.9" % "test",
164 | // Only allow one test at a time, even across projects, since they run in the same JVM
165 | parallelExecution in Test := false,
166 | // Make sure the test temp directory exists.
167 | resourceGenerators in Test <+= resourceManaged in Test map { outDir: File =>
168 | if (!new File(testTempDir).isDirectory()) {
169 | require(new File(testTempDir).mkdirs())
170 | }
171 | Seq[File]()
172 | },
173 | concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
174 | // Remove certain packages from Scaladoc
175 | scalacOptions in (Compile, doc) := Seq(
176 | "-groups",
177 | "-skip-packages", Seq(
178 | "akka",
179 | "org.apache.spark.api.python",
180 | "org.apache.spark.network",
181 | "org.apache.spark.deploy",
182 | "org.apache.spark.util.collection"
183 | ).mkString(":"),
184 | "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc"
185 | )
186 | )
187 |
188 | }
189 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/cloudera/spark/MemoryMonitor.scala:
--------------------------------------------------------------------------------
1 | // (c) Copyright 2018 Cloudera, Inc. All rights reserved.
2 | package com.cloudera.spark
3 |
4 | import java.lang.management._
5 | import java.math.{RoundingMode, MathContext}
6 | import java.text.SimpleDateFormat
7 | import java.util.Locale
8 | import java.util.concurrent.atomic.{AtomicInteger, AtomicBoolean, AtomicReference}
9 | import java.util.concurrent._
10 |
11 | import scala.collection.JavaConverters._
12 |
13 | import com.quantifind.sumac.FieldArgs
14 |
15 | import org.apache.spark.{TaskContext, SparkContext}
16 | import org.apache.spark.executor.ExecutorPlugin
17 | import org.apache.spark.memory.SparkMemoryManagerHandle
18 |
19 | class MemoryMonitor(val args: MemoryMonitorArgs) {
20 | val nettyMemoryHandle = SparkNettyMemoryHandle.get()
21 | val sparkMemManagerHandle = SparkMemoryManagerHandle.get()
22 | val memoryBean = ManagementFactory.getMemoryMXBean
23 | val poolBeans = ManagementFactory.getMemoryPoolMXBeans.asScala
24 | val offHeapPoolBeans = poolBeans.filter { pool =>
25 | // a heuristic which seems OK?
26 | !pool.isCollectionUsageThresholdSupported && pool.isUsageThresholdSupported
27 | }
28 | val memMgrBeans = ManagementFactory.getMemoryManagerMXBeans.asScala
29 | val bufferPoolsBeans = ManagementFactory.getPlatformMXBeans(classOf[BufferPoolMXBean]).asScala
30 |
31 |
32 | // TODO usageThresholds & collection usage thresholds
33 | // with that setup maybe I'd even just do this for every pool, not just offheap pools
34 |
35 | val getters: Seq[MemoryGetter] =
36 | Seq(new MemoryMxBeanGetter(memoryBean)) ++
37 | offHeapPoolBeans.map(new PoolGetter(_)) ++
38 | bufferPoolsBeans.map(new BufferPoolGetter(_)) ++
39 | nettyMemoryHandle.toSeq ++
40 | sparkMemManagerHandle.toSeq
41 |
42 | val namesAndReporting = getters.flatMap(_.namesAndReporting)
43 | val names = namesAndReporting.map(_._1)
44 | val reporting = namesAndReporting.map(_._2).toIndexedSeq
45 | val nMetrics = namesAndReporting.length
46 | val getterAndOffset = {
47 | var offset = 0
48 | getters.map { g =>
49 | val thisOffset = offset
50 | offset += g.namesAndReporting.length
51 | (g, thisOffset)
52 | }
53 | }
54 |
55 | val peakMemoryUsage = new MemoryPeaks(nMetrics)
56 | val peakUpdates = new PeakUpdate(nMetrics)
57 | val lastNonShutdownSnapshot = new AtomicReference[MemorySnapshot]()
58 | val lastThreadDump = new AtomicReference[Array[ThreadInfo]]()
59 | val inShutdown = new AtomicBoolean(false)
60 |
61 | def showMetricNames: Unit = {
62 | println(s"${nMetrics} Metrics")
63 | (0 until nMetrics).foreach { idx => println(names(idx))}
64 | }
65 |
66 | def collectSnapshot: MemorySnapshot = {
67 | val now = System.currentTimeMillis()
68 | val values = new Array[Long](nMetrics)
69 | getterAndOffset.foreach { case (g, offset) =>
70 | g.values(values, offset)
71 | }
72 | val s = new MemorySnapshot(now, values)
73 | if (!inShutdown.get()) {
74 | lastNonShutdownSnapshot.set(s)
75 | if (args.threadDumpEnabled) {
76 | lastThreadDump.set(MemoryMonitor.getThreadInfo)
77 | showLastThreadDump
78 | }
79 | }
80 | s
81 | }
82 |
83 | def showSnapshot(mem: MemorySnapshot): Unit = {
84 | println(s"Mem usage at ${MemoryMonitor.dateFormat.format(mem.time)}")
85 | println("===============")
86 | // TODO headers for each getter?
87 | (0 until nMetrics).foreach { idx =>
88 | val v = mem.values(idx)
89 | println(names(idx) + "\t:" + MemoryMonitor.bytesToString(v) + "(" + v + ")")
90 | }
91 | println()
92 | println()
93 | }
94 |
95 | def updateAndMaybeShowPeaks(): Unit = {
96 | val snapshot = collectSnapshot
97 | if (peakMemoryUsage.update(snapshot, peakUpdates, reporting)) {
98 | showUpdates(snapshot.time, peakMemoryUsage, peakUpdates)
99 | }
100 | }
101 |
102 | def showUpdates(time: Long, peakMemory: MemoryPeaks, updates: PeakUpdate): Unit = {
103 | println(s"Peak Memory updates:${MemoryMonitor.dateFormat.format(time)}")
104 | (0 until updates.nUpdates).foreach { updateIdx =>
105 | val metricIdx = updates.updateIdx(updateIdx)
106 | val name = names(metricIdx)
107 | val currentVal = MemoryMonitor.bytesToString(peakMemoryUsage.values(metricIdx))
108 | val rawDelta = updates.delta(updateIdx)
109 | val delta = (if (rawDelta > 0) "+" else "-") + MemoryMonitor.bytesToString(rawDelta)
110 | println(s"$name\t:\t$currentVal ($delta)")
111 | }
112 | }
113 |
114 | def showPeaks(time: Long): Unit = {
115 | println(s"Peak Memory usage so far ${MemoryMonitor.dateFormat.format(time)}")
116 | // TODO headers for each getter?
117 | (0 until nMetrics).foreach { idx =>
118 | println(names(idx) + "\t:" + MemoryMonitor.bytesToString(peakMemoryUsage.values(idx)) +
119 | "\t\t\t\t" + MemoryMonitor.dateFormat.format(peakMemoryUsage.peakTimes(idx)))
120 | }
121 | }
122 |
123 | def showCurrentMemUsage: Unit = {
124 | showSnapshot(collectSnapshot)
125 | }
126 |
127 | def showLastThreadDump: Unit = {
128 | val threads = lastThreadDump.get()
129 | if (threads != null) {
130 | println("last thread dump:")
131 | MemoryMonitor.showThreadDump(threads)
132 | }
133 | }
134 |
135 | def installShutdownHook: Unit = {
136 | Runtime.getRuntime.addShutdownHook(new Thread(){
137 | override def run(): Unit = {
138 | inShutdown.set(true)
139 | println()
140 | println("IN SHUTDOWN")
141 | println("================")
142 | val snapshot = collectSnapshot
143 | showSnapshot(snapshot)
144 | peakMemoryUsage.update(snapshot, peakUpdates, reporting)
145 | showPeaks(snapshot.time)
146 | println("Last non-shutdown snapshot:")
147 | showSnapshot(lastNonShutdownSnapshot.get())
148 |
149 | showLastThreadDump
150 | }
151 | })
152 | }
153 |
154 | def beanInfo(): Unit = {
155 |
156 | memMgrBeans.foreach { mgr =>
157 | println(mgr.getName + " is managing " + mgr.getMemoryPoolNames.mkString(","))
158 | }
159 |
160 | poolBeans.foreach { pool =>
161 | println(pool.getName())
162 | println("============")
163 | println(pool.getName() + " is managed by " + pool.getMemoryManagerNames.mkString(","))
164 | if (pool.isUsageThresholdSupported)
165 | println("supports usage threshold")
166 | if (pool.isCollectionUsageThresholdSupported)
167 | println("supports collection usage threshold")
168 | pool.getUsage
169 | println()
170 | println()
171 | }
172 |
173 | println("BUFFER POOLS")
174 | bufferPoolsBeans.foreach { bp =>
175 | println(s"${bp.getName}: ${bp.getMemoryUsed} / ${bp.getTotalCapacity}")
176 | }
177 | }
178 | }
179 |
180 | object MemoryMonitor {
181 |
182 | val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS")
183 |
184 | private var monitor: MemoryMonitor = null
185 | private var shutdownHookInstalled = false
186 | private var scheduler: ScheduledThreadPoolExecutor = _
187 | def install(args: MemoryMonitorArgs): MemoryMonitor = synchronized {
188 | if (monitor == null) {
189 | monitor = new MemoryMonitor(args)
190 | }
191 | monitor
192 | }
193 |
194 | def installShutdownHook(): Unit = synchronized {
195 | if (!shutdownHookInstalled) {
196 | monitor.installShutdownHook
197 | shutdownHookInstalled = true
198 | }
199 | }
200 |
201 | def startPolling(args: MemoryMonitorArgs): Unit = synchronized {
202 | if (scheduler == null) {
203 | scheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactory {
204 | override def newThread(r: Runnable): Thread = {
205 | val t = new Thread(r, "memory-poll thread")
206 | t.setDaemon(true)
207 | t
208 | }
209 | })
210 | }
211 | scheduler.scheduleWithFixedDelay(new Runnable {
212 | override def run(): Unit = {
213 | if (args.showEverySnapshot) {
214 | monitor.showCurrentMemUsage
215 | } else {
216 | monitor.updateAndMaybeShowPeaks
217 | }
218 | }
219 | }, 0, args.freq.get, TimeUnit.MILLISECONDS)
220 | }
221 |
222 | def listAllMBeans: Unit = {
223 | val server = ManagementFactory.getPlatformMBeanServer
224 | val allBeans = server.queryNames(null, null)
225 | println("ALL BEANS")
226 | println("=============")
227 | allBeans.asScala.map{_.toString}.toArray.sorted.foreach { ob => println(ob) }
228 | println()
229 | println()
230 | }
231 |
232 | def showLimits: Unit = {
233 | println("sun.misc.VM.maxDirectMemory(): " + sun.misc.VM.maxDirectMemory())
234 | println("Runtime.getRuntime.maxMemory(): " + Runtime.getRuntime.maxMemory())
235 | }
236 |
237 | def installIfSysProps(): Unit = {
238 | val args = MemoryMonitorArgs.sysPropsArgs
239 | if (args.enabled) {
240 | install(args)
241 | installShutdownHook()
242 | args.freq.foreach { freq =>
243 | println(s"POLLING memory monitor every $freq millis")
244 | monitor.showCurrentMemUsage
245 | println("done with initial show")
246 | startPolling(args)
247 | }
248 | }
249 | }
250 |
251 | def installOnExecIfStaticAllocation(sc: SparkContext): Unit = {
252 | if (!SparkMemoryManagerHandle.isDynamicAllocation(sc)) {
253 | installOnExecutors(sc)
254 | } else {
255 | println ("********* WARNING ***** not installing on executors because of DA")
256 | }
257 | }
258 |
259 | def installOnExecutors(sc: SparkContext, numTasks: Int = -1, sleep: Long = 1): Unit = {
260 | assert(!SparkMemoryManagerHandle.isDynamicAllocation(sc))
261 | val t = if (numTasks == -1) {
262 | sc.getExecutorMemoryStatus.size * 2
263 | } else {
264 | numTasks
265 | }
266 | println(s"Running $t tasks to install memory monitor on executors")
267 | sc.parallelize(1 to t, t).foreach { _ =>
268 | Thread.sleep(sleep)
269 | installIfSysProps()
270 | }
271 | }
272 |
273 | /**
274 | * Convert a quantity in bytes to a human-readable string such as "4.0 MB".
275 | */
276 | def bytesToString(size: Long): String = bytesToString(BigInt(size))
277 |
278 | def bytesToString(size: BigInt): String = {
279 | val EB = 1L << 60
280 | val PB = 1L << 50
281 | val TB = 1L << 40
282 | val GB = 1L << 30
283 | val MB = 1L << 20
284 | val KB = 1L << 10
285 |
286 | if (size >= BigInt(1L << 11) * EB) {
287 | // The number is too large, show it in scientific notation.
288 | BigDecimal(size, new MathContext(3, RoundingMode.HALF_UP)).toString() + " B"
289 | } else {
290 | val (value, unit) = {
291 | if (size >= 2 * EB) {
292 | (BigDecimal(size) / EB, "EB")
293 | } else if (size >= 2 * PB) {
294 | (BigDecimal(size) / PB, "PB")
295 | } else if (size >= 2 * TB) {
296 | (BigDecimal(size) / TB, "TB")
297 | } else if (size >= 2 * GB) {
298 | (BigDecimal(size) / GB, "GB")
299 | } else if (size >= 2 * MB) {
300 | (BigDecimal(size) / MB, "MB")
301 | } else if (size >= 2 * KB) {
302 | (BigDecimal(size) / KB, "KB")
303 | } else {
304 | (BigDecimal(size), "B")
305 | }
306 | }
307 | "%.1f %s".formatLocal(Locale.US, value, unit)
308 | }
309 | }
310 |
311 | def getThreadInfo: Array[ThreadInfo] = {
312 | // I'm avoiding getting locks for the moment in a random hope that it might be faster,
313 | // and because I don't really care right now
314 | ManagementFactory.getThreadMXBean.dumpAllThreads(false, false)
315 | }
316 |
317 | def showThreadDump(threads: Array[ThreadInfo]): Unit = {
318 | threads.foreach { t =>
319 | if (t == null) {
320 | println("")
321 | } else {
322 | println(t.getThreadId + " " + t.getThreadName + " " + t.getThreadState)
323 | t.getStackTrace.foreach { elem => println("\t" + elem) }
324 | println()
325 | }
326 | }
327 | }
328 | }
329 |
330 | class MemoryMonitorExecutorExtension extends ExecutorPlugin {
331 | // the "extension class" api just lets you invoke a constructor. We really just want to
332 | // call this static method, so that's good enough.
333 | MemoryMonitor.installIfSysProps()
334 | val args = MemoryMonitorArgs.sysPropsArgs
335 |
336 | val monitoredTaskCount = new AtomicInteger(0)
337 |
338 | val scheduler = if (args.stagesToPoll != null && args.stagesToPoll.nonEmpty) {
339 | // TODO share polling executors?
340 | new ScheduledThreadPoolExecutor(1, new ThreadFactory {
341 | override def newThread(r: Runnable): Thread = {
342 | val t = new Thread(r, "thread-dump poll thread")
343 | t.setDaemon(true)
344 | t
345 | }
346 | })
347 | } else {
348 | null
349 | }
350 | val pollingTask = new AtomicReference[ScheduledFuture[_]]()
351 |
352 | override def taskStart(taskContext: TaskContext): Unit = {
353 | if (args.stagesToPoll.contains(taskContext.stageId())) {
354 | if (monitoredTaskCount.getAndIncrement() == 0) {
355 | // TODO schedule thread polling
356 | val task = scheduler.scheduleWithFixedDelay(new Runnable {
357 | override def run(): Unit = {
358 | val d = MemoryMonitor.dateFormat.format(System.currentTimeMillis())
359 | println(s"Polled thread dump @ $d")
360 | MemoryMonitor.showThreadDump(MemoryMonitor.getThreadInfo)
361 | }
362 | }, 0, args.threadDumpFreqMillis, TimeUnit.MILLISECONDS)
363 | pollingTask.set(task)
364 | }
365 | }
366 | }
367 |
368 | override def onTaskFailure(context: TaskContext, error: Throwable): Unit = {
369 | removeActiveTask(context)
370 | }
371 |
372 | override def onTaskCompletion(context: TaskContext): Unit = {
373 | removeActiveTask(context)
374 | }
375 |
376 | private def removeActiveTask(context: TaskContext): Unit = {
377 | if (args.stagesToPoll.contains(context.stageId())) {
378 | if (monitoredTaskCount.decrementAndGet() == 0) {
379 | pollingTask.get().cancel(false)
380 | }
381 | }
382 | }
383 | }
384 |
385 | class MemoryMonitorArgs extends FieldArgs {
386 | var enabled = false
387 | // java.lang.Long because scalac makes Option[Long] look like Option[Any] to java reflection
388 | var freq: Option[java.lang.Long] = None
389 | var showEverySnapshot = false
390 |
391 | var stagesToPoll: Array[Int] = _
392 |
393 | var threadDumpFreqMillis: Int = 1000
394 | var threadDumpEnabled = false
395 |
396 | var verbose = false
397 | }
398 |
399 | object MemoryMonitorArgs {
400 | val prefix = "memory.monitor."
401 | val prefixLen = prefix.length
402 |
403 | lazy val sysPropsArgs = {
404 | val args = new MemoryMonitorArgs
405 | args.parse(sys.props.collect { case (k,v) if k.startsWith(prefix) =>
406 | k.substring(prefixLen) -> v
407 | })
408 | if (args.stagesToPoll != null && args.stagesToPoll.nonEmpty) {
409 | System.out.println(s"will poll thread dumps for stages ${args.stagesToPoll.mkString(",")}")
410 | } else {
411 | args.stagesToPoll = Array()
412 | }
413 | args
414 | }
415 | }
416 |
--------------------------------------------------------------------------------