From 503acd3a379a3686d343fdf072fc231b8fba78f9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:47:27 +0800 Subject: [PATCH 01/22] Build metrics system framwork --- conf/metrics.properties | 13 ++++ .../metrics/AbstractInstrumentation.scala | 68 +++++++++++++++++++ .../scala/spark/metrics/MetricsConfig.scala | 55 +++++++++++++++ .../spark/metrics/sink/ConsoleSink.scala | 43 ++++++++++++ .../scala/spark/metrics/sink/CsvSink.scala | 53 +++++++++++++++ .../scala/spark/metrics/sink/JmxSink.scala | 17 +++++ .../main/scala/spark/metrics/sink/Sink.scala | 6 ++ 7 files changed, 255 insertions(+) create mode 100644 conf/metrics.properties create mode 100644 core/src/main/scala/spark/metrics/AbstractInstrumentation.scala create mode 100644 core/src/main/scala/spark/metrics/MetricsConfig.scala create mode 100644 core/src/main/scala/spark/metrics/sink/ConsoleSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/CsvSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/JmxSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/Sink.scala diff --git a/conf/metrics.properties b/conf/metrics.properties new file mode 100644 index 0000000000..78749cf381 --- /dev/null +++ b/conf/metrics.properties @@ -0,0 +1,13 @@ +# syntax: [prefix].[sink].[instance].[options] + +*.sink.console.period=10 + +*.sink.console.unit=second + +master.sink.console.period=10 + +master.sink.console.unit=second + +worker.sink.console.period=20 + +worker.sink.console.unit=second diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala new file mode 100644 index 0000000000..0fed608488 --- /dev/null +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -0,0 +1,68 @@ +package spark.metrics + +import scala.collection.mutable + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import spark.Logging +import spark.metrics.sink._ + +trait AbstractInstrumentation extends Logging { + initLogging() + + def registryHandler: MetricRegistry + def instance: String + + val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val metricsConfig = new MetricsConfig(confFile) + + val sinks = new mutable.ArrayBuffer[Sink] + + def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + + // Register JMX sink as a default sink + sinks += new JmxSink(registryHandler) + + // Register other sinks according to conf + sinkConfigs.foreach { kv => + val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { + AbstractInstrumentation.DEFAULT_SINKS(kv._1) + } else { + kv._2.getProperty("class") + } + try { + val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, registryHandler) + sinks += sink.asInstanceOf[Sink] + } catch { + case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + } + } + + sinks.foreach(_.registerSink) + } + + def unregisterSinks() { + sinks.foreach(_.unregisterSink) + } +} + +object AbstractInstrumentation { + val DEFAULT_SINKS = Map( + "console" -> "spark.metrics.sink.ConsoleSink", + "csv" -> "spark.metrics.sink.CsvSink") + + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + + val timeUnits = Map( + "millisecond" -> TimeUnit.MILLISECONDS, + "second" -> TimeUnit.SECONDS, + "minute" -> TimeUnit.MINUTES, + "hour" -> TimeUnit.HOURS, + "day" -> TimeUnit.DAYS) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala new file mode 100644 index 0000000000..0fec1988ea --- /dev/null +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -0,0 +1,55 @@ +package spark.metrics + +import java.util.Properties +import java.io.FileInputStream + +import scala.collection.mutable +import scala.util.matching.Regex + +class MetricsConfig(val configFile: String) { + val properties = new Properties() + var fis: FileInputStream = _ + + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + + val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) + if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + + def getInstance(inst: String) = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) + } + } +} + +object MetricsConfig { + val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_PREFIX = "*" + val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + + def subProperties(prop: Properties, regex: Regex) = { + val subProperties = new mutable.HashMap[String, Properties] + + import scala.collection.JavaConversions._ + prop.foreach { kv => + val regex(a, b) = kv._1 + subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) + println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + } + + subProperties + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala new file mode 100644 index 0000000000..5426af8c4c --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -0,0 +1,43 @@ +package spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + } + + var reporter: ConsoleReporter = _ + + override def registerSink() { + reporter = ConsoleReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object ConsoleSink { + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala new file mode 100644 index 0000000000..3a80c36901 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -0,0 +1,53 @@ +package spark.metrics.sink + +import java.io.File +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{CsvReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + } + + val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + case Some(s) => s + case None => CsvSink.CSV_DEFAULT_DIR + } + + var reporter: CsvReporter = _ + + override def registerSink() { + reporter = CsvReporter.forRegistry(registry) + .formatFor(Locale.US) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build(new File(pollDir)) + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object CsvSink { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "1" + val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_DIR = "/tmp/" +} + diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala new file mode 100644 index 0000000000..56e5677700 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -0,0 +1,17 @@ +package spark.metrics.sink + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +class JmxSink(registry: MetricRegistry) extends Sink { + var reporter: JmxReporter = _ + + override def registerSink() { + reporter = JmxReporter.forRegistry(registry).build() + reporter.start() + } + + override def unregisterSink() { + reporter.stop() + } + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala new file mode 100644 index 0000000000..65ebcb4eac --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -0,0 +1,6 @@ +package spark.metrics.sink + +trait Sink { + def registerSink: Unit + def unregisterSink: Unit +} \ No newline at end of file From 9dec8c73e6f0c3b6b55a11ff92cc9bff18dadd24 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:48:41 +0800 Subject: [PATCH 02/22] Add Master and Worker instrumentation support --- .../scala/spark/deploy/master/Master.scala | 9 ++- .../deploy/master/MasterInstrumentation.scala | 44 +++++++++++++++ .../scala/spark/deploy/worker/Worker.scala | 11 +++- .../deploy/worker/WorkerInstrumentation.scala | 55 +++++++++++++++++++ 4 files changed, 115 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index eddcafd84d..3a7c4e5a52 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -33,7 +33,8 @@ import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor +with Logging with MasterInstrumentation { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -73,6 +74,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + + initialize(this) } override def postStop() { @@ -316,6 +319,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + uninitialize() + } } private[spark] object Master { diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala new file mode 100644 index 0000000000..13088189a4 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -0,0 +1,44 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait MasterInstrumentation extends AbstractInstrumentation { + var masterInst: Option[Master] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "master" + + def initialize(master: Master) { + masterInst = Some(master) + + // Register and start all the sinks + registerSinks + } + + def uninitialize() { + unregisterSinks + } + + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0bd88ea253..b64bdb8d28 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -41,7 +41,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging { + extends Actor with Logging with WorkerInstrumentation { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -97,6 +97,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() + startWebUi() + + initialize(this) } def connectToMaster() { @@ -155,10 +158,10 @@ private[spark] class Worker( case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() - + case RequestWorkerState => { sender ! WorkerState(host, port, workerId, executors.values.toList, - finishedExecutors.values.toList, masterUrl, cores, memory, + finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } } @@ -178,6 +181,8 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() + + uninitialize() } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala new file mode 100644 index 0000000000..04c43ce33b --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -0,0 +1,55 @@ +package spark.deploy.worker + +import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { + var workerInst: Option[Worker] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "worker" + + def initialize(worker: Worker) { + workerInst = Some(worker) + + // Register and start all the sinks + registerSinks() + } + + def uninitialize() { + unregisterSinks() + } + + // Gauge for executors number + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) + }) +} \ No newline at end of file From c3daad3f65630eb4ed536d06c0d467cde57a8142 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 12:00:19 +0800 Subject: [PATCH 03/22] Update metric source support for instrumentation --- conf/metrics.properties | 4 ++- .../deploy/master/MasterInstrumentation.scala | 7 +++-- .../deploy/worker/WorkerInstrumentation.scala | 5 +++- .../metrics/AbstractInstrumentation.scala | 29 +++++++++++++++++-- .../scala/spark/metrics/MetricsConfig.scala | 12 ++++---- .../scala/spark/metrics/sink/CsvSink.scala | 4 +-- .../main/scala/spark/metrics/sink/Sink.scala | 1 + .../spark/metrics/source/JvmSource.scala | 17 +++++++++++ .../scala/spark/metrics/source/Source.scala | 5 ++++ 9 files changed, 70 insertions(+), 14 deletions(-) create mode 100644 core/src/main/scala/spark/metrics/source/JvmSource.scala create mode 100644 core/src/main/scala/spark/metrics/source/Source.scala diff --git a/conf/metrics.properties b/conf/metrics.properties index 78749cf381..0bbb6b5229 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,9 +1,11 @@ -# syntax: [prefix].[sink].[instance].[options] +# syntax: [prefix].[sink|source].[instance].[options] *.sink.console.period=10 *.sink.console.unit=second +master.source.jvm.class=spark.metrics.source.JvmSource + master.sink.console.period=10 master.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 13088189a4..c295e725d7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -15,12 +15,15 @@ private[spark] trait MasterInstrumentation extends AbstractInstrumentation { def initialize(master: Master) { masterInst = Some(master) + // Register all the sources + registerSources() + // Register and start all the sinks - registerSinks + registerSinks() } def uninitialize() { - unregisterSinks + unregisterSinks() } // Gauge for worker numbers in cluster diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 04c43ce33b..2f725300b5 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -15,6 +15,9 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { def initialize(worker: Worker) { workerInst = Some(worker) + // Register all the sources + registerSources() + // Register and start all the sinks registerSinks() } @@ -36,7 +39,7 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) }) diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala index 0fed608488..9cae1e0220 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -9,17 +9,39 @@ import java.util.concurrent.TimeUnit import spark.Logging import spark.metrics.sink._ +import spark.metrics.source._ -trait AbstractInstrumentation extends Logging { +private [spark] trait AbstractInstrumentation extends Logging { initLogging() + // Get MetricRegistry handler def registryHandler: MetricRegistry + // Get the instance name def instance: String val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] + val sources = new mutable.ArrayBuffer[Source] + + def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + + // Register all the sources + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) + .newInstance(registryHandler) + sources += source.asInstanceOf[Source] + } catch { + case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + } + } + sources.foreach(_.registerSource) + } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) @@ -33,6 +55,7 @@ trait AbstractInstrumentation extends Logging { val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { AbstractInstrumentation.DEFAULT_SINKS(kv._1) } else { + // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { @@ -40,10 +63,9 @@ trait AbstractInstrumentation extends Logging { .newInstance(kv._2, registryHandler) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) } @@ -58,6 +80,7 @@ object AbstractInstrumentation { "csv" -> "spark.metrics.sink.CsvSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r val timeUnits = Map( "millisecond" -> TimeUnit.MILLISECONDS, diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 0fec1988ea..be4f670918 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -6,7 +6,7 @@ import java.io.FileInputStream import scala.collection.mutable import scala.util.matching.Regex -class MetricsConfig(val configFile: String) { +private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() var fis: FileInputStream = _ @@ -36,7 +36,7 @@ class MetricsConfig(val configFile: String) { } object MetricsConfig { - val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_CONFIG_FILE = "conf/metrics.properties" val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r @@ -45,9 +45,11 @@ object MetricsConfig { import scala.collection.JavaConversions._ prop.foreach { kv => - val regex(a, b) = kv._1 - subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) - println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + if (regex.findPrefixOf(kv._1) != None) { + val regex(prefix, suffix) = kv._1 + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) + println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) + } } subProperties diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3a80c36901..1d663f6cff 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -46,8 +46,8 @@ object CsvSink { val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "1" - val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" val CSV_DEFAULT_DIR = "/tmp/" } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 65ebcb4eac..26052b7231 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,5 +2,6 @@ package spark.metrics.sink trait Sink { def registerSink: Unit + def unregisterSink: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala new file mode 100644 index 0000000000..8f6bf48843 --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -0,0 +1,17 @@ +package spark.metrics.source + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} + +class JvmSource(registry: MetricRegistry) extends Source { + // Initialize memory usage gauge for jvm + val memUsageMetricSet = new MemoryUsageGaugeSet + + // Initialize garbage collection usage gauge for jvm + val gcMetricSet = new GarbageCollectorMetricSet + + override def registerSource() { + registry.registerAll(memUsageMetricSet) + registry.registerAll(gcMetricSet) + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala new file mode 100644 index 0000000000..35cfe0c8ff --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -0,0 +1,5 @@ +package spark.metrics.source + +trait Source { + def registerSource: Unit +} \ No newline at end of file From 03f9871116801abbdd7b4c7892c8d6affb1c4d9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 18:29:54 +0800 Subject: [PATCH 04/22] MetricsSystem refactor --- .../scala/spark/deploy/master/Master.scala | 17 ++- .../deploy/master/MasterInstrumentation.scala | 66 ++++----- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../deploy/worker/WorkerInstrumentation.scala | 131 +++++++++++------- ...trumentation.scala => MetricsSystem.scala} | 54 +++++--- .../spark/metrics/sink/ConsoleSink.scala | 10 +- .../scala/spark/metrics/sink/CsvSink.scala | 10 +- .../scala/spark/metrics/sink/JmxSink.scala | 4 +- .../main/scala/spark/metrics/sink/Sink.scala | 4 +- .../spark/metrics/source/JvmSource.scala | 30 ++-- .../scala/spark/metrics/source/Source.scala | 6 +- 11 files changed, 194 insertions(+), 150 deletions(-) rename core/src/main/scala/spark/metrics/{AbstractInstrumentation.scala => MetricsSystem.scala} (64%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3a7c4e5a52..e44f5e3168 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} +import spark.metrics.MetricsSystem import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor -with Logging with MasterInstrumentation { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -57,6 +57,8 @@ with Logging with MasterInstrumentation { val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") + + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -75,7 +77,7 @@ with Logging with MasterInstrumentation { webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - initialize(this) + Master.metricsSystem.registerSource(masterInstrumentation) } override def postStop() { @@ -319,21 +321,22 @@ with Logging with MasterInstrumentation { removeWorker(worker) } } - - override def postStop() { - uninitialize() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + + private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + + metricsSystem.start() actorSystem.awaitTermination() + metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index c295e725d7..5ea9a90319 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,47 +1,35 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} +import java.util.{Map, HashMap => JHashMap} -import spark.metrics.AbstractInstrumentation +import com.codahale.metrics.{Gauge, Metric} -private[spark] trait MasterInstrumentation extends AbstractInstrumentation { - var masterInst: Option[Master] = None - val metricRegistry = new MetricRegistry() - - override def registryHandler = metricRegistry - - override def instance = "master" - - def initialize(master: Master) { - masterInst = Some(master) +import spark.metrics.source.Source + +private[spark] class MasterInstrumentation(val master: Master) extends Source { + val className = classOf[Master].getName() + val instrumentationName = "master" - // Register all the sources - registerSources() + override def sourceName = instrumentationName + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for worker numbers in cluster + gauges.put(className + ".workers.number", new Gauge[Int] { + override def getValue: Int = master.workers.size + }) + + // Gauge for application numbers in cluster + gauges.put(className + ".apps.number", new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) - }) - } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b64bdb8d28..eaa1c1806f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import akka.util.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ +import spark.metrics.MetricsSystem import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date @@ -41,7 +42,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging with WorkerInstrumentation { + extends Actor with Logging { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -67,6 +68,8 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val workerInstrumentation = new WorkerInstrumentation(this) + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -99,7 +102,8 @@ private[spark] class Worker( connectToMaster() startWebUi() - initialize(this) + Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.start() } def connectToMaster() { @@ -182,11 +186,13 @@ private[spark] class Worker( executors.values.foreach(_.kill()) webUi.stop() - uninitialize() + Worker.metricsSystem.stop() } } private[spark] object Worker { + private val metricsSystem = MetricsSystem.createMetricsSystem("worker") + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 2f725300b5..37fd154859 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,58 +1,89 @@ package spark.deploy.worker -import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} +import com.codahale.metrics.{Gauge, Metric} -import spark.metrics.AbstractInstrumentation +import java.util.{Map, HashMap => JHashMap} -private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { - var workerInst: Option[Worker] = None - val metricRegistry = new MetricRegistry() +import spark.metrics.source.Source + +private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { + val className = classOf[Worker].getName() - override def registryHandler = metricRegistry - - override def instance = "worker" - - def initialize(worker: Worker) { - workerInst = Some(worker) + override def sourceName = "worker" - // Register all the sources - registerSources() + override def getMetrics: Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for executors number + gauges.put(className + ".executor.number", new Gauge[Int]{ + override def getValue: Int = worker.executors.size + }) + + gauges.put(className + ".core_used.number", new Gauge[Int]{ + override def getValue: Int = worker.coresUsed + }) + + gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryUsed + }) + + gauges.put(className + ".core_free.number", new Gauge[Int]{ + override def getValue: Int = worker.coresFree + }) + + gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryFree + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for executors number - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) - }) - - // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) - }) - - // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) - }) -} \ No newline at end of file +} +//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { +// var workerInst: Option[Worker] = None +// val metricRegistry = new MetricRegistry() +// +// override def registryHandler = metricRegistry +// +// override def instance = "worker" +// +// def initialize(worker: Worker) { +// workerInst = Some(worker) +// +// registerSources() +// registerSinks() +// } +// +// def uninitialize() { +// unregisterSinks() +// } +// +// // Gauge for executors number +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) +// }) +// +// // Gauge for cores used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) +// }) +// +// // Gauge for cores free of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) +// }) +//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala similarity index 64% rename from core/src/main/scala/spark/metrics/AbstractInstrumentation.scala rename to core/src/main/scala/spark/metrics/MetricsSystem.scala index 9cae1e0220..ea1bc490b5 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -2,7 +2,7 @@ package spark.metrics import scala.collection.mutable -import com.codahale.metrics.{JmxReporter, MetricRegistry} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -11,70 +11,76 @@ import spark.Logging import spark.metrics.sink._ import spark.metrics.source._ -private [spark] trait AbstractInstrumentation extends Logging { +private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - // Get MetricRegistry handler - def registryHandler: MetricRegistry - // Get the instance name - def instance: String - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + def start() { + registerSources() + registerSinks() + } + + def stop() { + sinks.foreach(_.stop) + } + + def registerSource(source: Source) { + sources += source + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + } + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - // Register all the sources + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) - .newInstance(registryHandler) + val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } } - sources.foreach(_.registerSource) } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(registryHandler) + sinks += new JmxSink(MetricsSystem.registry) // Register other sinks according to conf sinkConfigs.foreach { kv => - val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { - AbstractInstrumentation.DEFAULT_SINKS(kv._1) + val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { + MetricsSystem.DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, registryHandler) + .newInstance(kv._2, MetricsSystem.registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) - } - - def unregisterSinks() { - sinks.foreach(_.unregisterSink) + sinks.foreach(_.start) } } -object AbstractInstrumentation { +private[spark] object MetricsSystem { + val registry = new MetricRegistry() + val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -88,4 +94,6 @@ object AbstractInstrumentation { "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) + + def createMetricsSystem(instance: String) = new MetricsSystem(instance) } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 5426af8c4c..b49a211fb3 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { @@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend } val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } var reporter: ConsoleReporter = _ - override def registerSink() { + override def start() { reporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) @@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 1d663f6cff..3f572c8e05 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { @@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si } val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) } val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { @@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si var reporter: CsvReporter = _ - override def registerSink() { + override def start() { reporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 56e5677700..e223dc26e9 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { var reporter: JmxReporter = _ - override def registerSink() { + override def start() { reporter = JmxReporter.forRegistry(registry).build() reporter.start() } - override def unregisterSink() { + override def stop() { reporter.stop() } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 26052b7231..9fef894fde 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -1,7 +1,7 @@ package spark.metrics.sink trait Sink { - def registerSink: Unit + def start: Unit - def unregisterSink: Unit + def stop: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f6bf48843..7a7c1b6ffb 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,23 @@ package spark.metrics.source -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} +import java.util.{Map, HashMap => JHashMap} -class JvmSource(registry: MetricRegistry) extends Source { - // Initialize memory usage gauge for jvm - val memUsageMetricSet = new MemoryUsageGaugeSet - - // Initialize garbage collection usage gauge for jvm - val gcMetricSet = new GarbageCollectorMetricSet - - override def registerSource() { - registry.registerAll(memUsageMetricSet) - registry.registerAll(gcMetricSet) +import com.codahale.metrics.Metric +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} + +class JvmSource extends Source { + override def sourceName = "jvm" + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] + + import scala.collection.JavaConversions._ + val gcMetricSet = new GarbageCollectorMetricSet + gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + val memGaugeSet = new MemoryUsageGaugeSet + memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + gauges } } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 35cfe0c8ff..edd59de46a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,5 +1,7 @@ package spark.metrics.source -trait Source { - def registerSource: Unit +import com.codahale.metrics.MetricSet + +trait Source extends MetricSet { + def sourceName: String } \ No newline at end of file From 4d6dd67fa1f8f031f1ef46a442cec733fa3b1a7a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 28 Jun 2013 02:26:55 +0800 Subject: [PATCH 05/22] refactor metrics system 1.change source abstract class to support MetricRegistry 2.change master/work/jvm source class --- .../deploy/master/MasterInstrumentation.scala | 44 ++++----- .../deploy/worker/WorkerInstrumentation.scala | 98 +++++-------------- .../scala/spark/metrics/MetricsSystem.scala | 23 ++--- .../spark/metrics/sink/ConsoleSink.scala | 11 +-- .../scala/spark/metrics/sink/CsvSink.scala | 9 +- .../scala/spark/metrics/sink/JmxSink.scala | 5 +- .../spark/metrics/source/JvmSource.scala | 22 ++--- .../scala/spark/metrics/source/Source.scala | 6 +- 8 files changed, 84 insertions(+), 134 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 5ea9a90319..46c90b94d2 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -4,32 +4,32 @@ import java.util.{Map, HashMap => JHashMap} import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} + import spark.metrics.source.Source +import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { val className = classOf[Master].getName() val instrumentationName = "master" - - override def sourceName = instrumentationName - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for worker numbers in cluster - gauges.put(className + ".workers.number", new Gauge[Int] { + val metricRegistry = new MetricRegistry() + val sourceName = instrumentationName + + metricRegistry.register(MetricRegistry.name("workers","number"), + new Gauge[Int] { override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - gauges.put(className + ".apps.number", new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps", "number"), + new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), + new Gauge[Int] { override def getValue: Int = master.waitingApps.size - }) - - gauges - } -} \ No newline at end of file + }) + +} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 37fd154859..5ce29cf04c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -4,86 +4,42 @@ import com.codahale.metrics.{Gauge, Metric} import java.util.{Map, HashMap => JHashMap} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val className = classOf[Worker].getName() - override def sourceName = "worker" - - override def getMetrics: Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for executors number - gauges.put(className + ".executor.number", new Gauge[Int]{ + val sourceName = "worker" + + val metricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { override def getValue: Int = worker.executors.size - }) - - gauges.put(className + ".core_used.number", new Gauge[Int]{ + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresUsed - }) - - gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryUsed - }) - - gauges.put(className + ".core_free.number", new Gauge[Int]{ + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresFree - }) - - gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryFree }) - - gauges - } } -//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { -// var workerInst: Option[Worker] = None -// val metricRegistry = new MetricRegistry() -// -// override def registryHandler = metricRegistry -// -// override def instance = "worker" -// -// def initialize(worker: Worker) { -// workerInst = Some(worker) -// -// registerSources() -// registerSinks() -// } -// -// def uninitialize() { -// unregisterSinks() -// } -// -// // Gauge for executors number -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) -// }) -// -// // Gauge for cores used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) -// }) -// -// // Gauge for cores free of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) -// }) -//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index ea1bc490b5..a23ccd2692 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,6 +5,7 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties +//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -19,10 +20,13 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + var registry = new MetricRegistry() + + registerSources() + registerSinks() def start() { - registerSources() - registerSinks() + sinks.foreach(_.start) } def stop() { @@ -31,20 +35,20 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registry.register(source.sourceName,source.metricRegistry) } def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } @@ -56,7 +60,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(MetricsSystem.registry) + sinks += new JmxSink(registry) // Register other sinks according to conf sinkConfigs.foreach { kv => @@ -68,19 +72,16 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, MetricsSystem.registry) + .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.start) } } private[spark] object MetricsSystem { - val registry = new MetricRegistry() - val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -96,4 +97,4 @@ private[spark] object MetricsSystem { "day" -> TimeUnit.DAYS) def createMetricsSystem(instance: String) = new MetricsSystem(instance) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index b49a211fb3..9cd17556fa 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -18,14 +18,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = _ - - override def start() { - reporter = ConsoleReporter.forRegistry(registry) + var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() - + + + override def start() { reporter.start(pollPeriod, pollUnit) } @@ -40,4 +39,4 @@ object ConsoleSink { val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3f572c8e05..62e51be0dc 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -24,15 +24,14 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = _ - - override def start() { - reporter = CsvReporter.forRegistry(registry) + var reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - + + + override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index e223dc26e9..98b55f7b7f 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -3,10 +3,9 @@ package spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = _ + var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { - reporter = JmxReporter.forRegistry(registry).build() reporter.start() } @@ -14,4 +13,4 @@ class JmxSink(registry: MetricRegistry) extends Sink { reporter.stop() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 7a7c1b6ffb..13270dae3c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -2,22 +2,16 @@ package spark.metrics.source import java.util.{Map, HashMap => JHashMap} -import com.codahale.metrics.Metric +import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} class JvmSource extends Source { - override def sourceName = "jvm" - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - import scala.collection.JavaConversions._ + val sourceName = "jvm" + val metricRegistry = new MetricRegistry() + val gcMetricSet = new GarbageCollectorMetricSet - gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - val memGaugeSet = new MemoryUsageGaugeSet - memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - - gauges - } -} \ No newline at end of file + + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) +} diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index edd59de46a..17cbe2f85a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,7 +1,9 @@ package spark.metrics.source import com.codahale.metrics.MetricSet +import com.codahale.metrics.MetricRegistry -trait Source extends MetricSet { +trait Source { def sourceName: String -} \ No newline at end of file + def metricRegistry: MetricRegistry +} From 7fb574bf666661fdf8a786de779f85efe2f15f0c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:14:30 +0800 Subject: [PATCH 06/22] Code clean and remarshal --- .../scala/spark/deploy/master/Master.scala | 9 +++++---- .../deploy/master/MasterInstrumentation.scala | 11 ++--------- .../deploy/worker/WorkerInstrumentation.scala | 18 ++++++------------ .../scala/spark/metrics/MetricsSystem.scala | 5 ++--- .../scala/spark/metrics/sink/ConsoleSink.scala | 1 - .../scala/spark/metrics/sink/CsvSink.scala | 1 - .../main/scala/spark/metrics/sink/Sink.scala | 1 - 7 files changed, 15 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e44f5e3168..cc0b2d4295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -78,6 +78,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.start() } override def postStop() { @@ -321,22 +322,22 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + Master.metricsSystem.stop() + } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) - - metricsSystem.start() actorSystem.awaitTermination() - metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 46c90b94d2..61a561c955 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,19 +1,12 @@ package spark.deploy.master -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{Gauge, Metric} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { - val className = classOf[Master].getName() - val instrumentationName = "master" val metricRegistry = new MetricRegistry() - val sourceName = instrumentationName + val sourceName = "master" metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 5ce29cf04c..94c20a98c1 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,44 +1,38 @@ package spark.deploy.worker -import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{Gauge, MetricRegistry} -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { - val className = classOf[Worker].getName() - val sourceName = "worker" - val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index a23ccd2692..5bfdc00eaf 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,7 +5,6 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties -//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -20,7 +19,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] - var registry = new MetricRegistry() + val registry = new MetricRegistry() registerSources() registerSinks() @@ -35,7 +34,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName,source.metricRegistry) + registry.register(source.sourceName, source.metricRegistry) } def registerSources() { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 9cd17556fa..e2e4197d1d 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -23,7 +23,6 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend .convertRatesTo(TimeUnit.SECONDS) .build() - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 62e51be0dc..c2d645331c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -30,7 +30,6 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 9fef894fde..3ffdcbdaba 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,6 +2,5 @@ package spark.metrics.sink trait Sink { def start: Unit - def stop: Unit } \ No newline at end of file From 576528f0f916cc0434972cbbc9321e4dcec45628 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:25:48 +0800 Subject: [PATCH 07/22] Add dependency of Codahale's metrics library --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f3f67b57c8..825319d3de 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -180,6 +180,8 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "com.codahale.metrics" % "metrics-core" % "3.0.0", + "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 871bc1687eaeb59df24b4778c5992a5f7f105cc8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:46:24 +0800 Subject: [PATCH 08/22] Add Executor instrumentation --- .../executor/ExecutorInstrumentation.scala | 35 +++++++++++++++++++ .../spark/executor/MesosExecutorBackend.scala | 10 +++++- .../executor/StandaloneExecutorBackend.scala | 12 +++++++ 3 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/executor/ExecutorInstrumentation.scala diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala new file mode 100644 index 0000000000..80aadb66b0 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -0,0 +1,35 @@ +package spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ + val metricRegistry = new MetricRegistry() + val sourceName = "executor" + + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), + new Gauge[Long] { + override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 4961c42fad..8b6ab0c391 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState +import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -32,6 +33,9 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) + MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -79,13 +83,17 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) {} + override def shutdown(d: ExecutorDriver) { + MesosExecutorBackend.metricsSystem.stop() + } } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..6ef74cd2ff 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} +import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -45,6 +46,8 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -52,6 +55,9 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing + + StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) + StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -81,9 +87,15 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } + + override def postStop() { + StandaloneExecutorBackend.metricsSystem.stop() + } } private[spark] object StandaloneExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From 5ce5dc9fcd7acf5c58dd3d456a629b01d57514e4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:48:21 +0800 Subject: [PATCH 09/22] Add default properties to deal with no configure file situation --- .../scala/spark/metrics/MetricsConfig.scala | 28 ++++++++++++------- .../scala/spark/metrics/MetricsSystem.scala | 9 ++---- .../spark/metrics/sink/ConsoleSink.scala | 6 ++-- .../scala/spark/metrics/sink/CsvSink.scala | 6 ++-- .../scala/spark/metrics/sink/JmxSink.scala | 6 ++-- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index be4f670918..7405192058 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,20 +1,25 @@ package spark.metrics import java.util.Properties -import java.io.FileInputStream +import java.io.{File, FileInputStream} import scala.collection.mutable import scala.util.matching.Regex private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - var fis: FileInputStream = _ + // Add default properties in case there's no properties file + MetricsConfig.setDefaultProperties(properties) - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } } val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) @@ -35,11 +40,15 @@ private [spark] class MetricsConfig(val configFile: String) { } } -object MetricsConfig { - val DEFAULT_CONFIG_FILE = "conf/metrics.properties" +private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + def setDefaultProperties(prop: Properties) { + prop.setProperty("*.sink.jmx.enabled", "default") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + } + def subProperties(prop: Properties, regex: Regex) = { val subProperties = new mutable.HashMap[String, Properties] @@ -48,7 +57,6 @@ object MetricsConfig { if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) - println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5bfdc00eaf..6e448cb2a5 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -14,7 +14,7 @@ import spark.metrics.source._ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] @@ -58,9 +58,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register JMX sink as a default sink - sinks += new JmxSink(registry) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { @@ -81,9 +78,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map( - "console" -> "spark.metrics.sink.ConsoleSink", - "csv" -> "spark.metrics.sink.CsvSink") + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index e2e4197d1d..d7b7a9e501 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -1,10 +1,10 @@ package spark.metrics.sink +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + import java.util.Properties import java.util.concurrent.TimeUnit -import com.codahale.metrics.{ConsoleReporter, MetricRegistry} - import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -18,7 +18,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index c2d645331c..e6c5bffd3c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -1,11 +1,11 @@ package spark.metrics.sink +import com.codahale.metrics.{CsvReporter, MetricRegistry} + import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import com.codahale.metrics.{CsvReporter, MetricRegistry} - import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -24,7 +24,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = CsvReporter.forRegistry(registry) + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 98b55f7b7f..f097a631c0 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,9 +1,11 @@ package spark.metrics.sink +import java.util.Properties + import com.codahale.metrics.{JmxReporter, MetricRegistry} -class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() +class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { reporter.start() From e080588f7396d9612ea5d909e59f2364c139103a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:04:06 +0800 Subject: [PATCH 10/22] Add metrics system unit test --- .../spark/metrics/MetricsConfigSuite.scala | 92 +++++++++++++++++++ .../spark/metrics/MetricsSystemSuite.scala | 65 +++++++++++++ 2 files changed, 157 insertions(+) create mode 100644 core/src/test/scala/spark/metrics/MetricsConfigSuite.scala create mode 100644 core/src/test/scala/spark/metrics/MetricsSystemSuite.scala diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala new file mode 100644 index 0000000000..0c7142c418 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -0,0 +1,92 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsConfigSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val prop = new Properties() + + prop.setProperty("*.sink.console.period", "10") + prop.setProperty("*.sink.console.unit", "second") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + prop.setProperty("master.sink.console.period", "20") + prop.setProperty("master.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + prop.store(os, "for test") + os.close() + } + + test("MetricsConfig with default properties") { + val conf = new MetricsConfig("dummy-file") + assert(conf.properties.size() === 2) + assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") + assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.getProperty("test-for-dummy") === null) + + val property = conf.getInstance("random") + assert(property.size() === 2) + assert(property.getProperty("sink.jmx.enabled") === "default") + assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + } + + test("MetricsConfig with properties set") { + val conf = new MetricsConfig(filePath) + + val masterProp = conf.getInstance("master") + assert(masterProp.size() === 4) + assert(masterProp.getProperty("sink.console.period") === "20") + assert(masterProp.getProperty("sink.console.unit") === "minute") + assert(masterProp.getProperty("sink.jmx.enabled") === "default") + assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + + val workerProp = conf.getInstance("worker") + assert(workerProp.size() === 4) + assert(workerProp.getProperty("sink.console.period") === "10") + assert(workerProp.getProperty("sink.console.unit") === "second") + } + + test("MetricsConfig with subProperties") { + val conf = new MetricsConfig(filePath) + + val propCategories = conf.propertyCategories + assert(propCategories.size === 2) + + val masterProp = conf.getInstance("master") + val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + assert(sourceProps.size === 1) + assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") + + val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + assert(sinkProps.size === 2) + assert(sinkProps.contains("console")) + assert(sinkProps.contains("jmx")) + + val consoleProps = sinkProps("console") + assert(consoleProps.size() === 2) + + val jmxProps = sinkProps("jmx") + assert(jmxProps.size() === 1) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala new file mode 100644 index 0000000000..5e8f8fcf80 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -0,0 +1,65 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsSystemSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val props = new Properties() + props.setProperty("*.sink.console.period", "10") + props.setProperty("*.sink.console.unit", "second") + props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") + props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") + props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") + props.setProperty("test.sink.console.period", "20") + props.setProperty("test.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + props.store(os, "for test") + os.close() + System.setProperty("spark.metrics.conf.file", filePath) + } + + test("MetricsSystem with default config") { + val metricsSystem = MetricsSystem.createMetricsSystem("default") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 1) + assert(sources(0).sourceName === "jvm") + } + + test("MetricsSystem with sources add") { + val metricsSystem = MetricsSystem.createMetricsSystem("test") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 2) + + val source = new spark.deploy.master.MasterInstrumentation(null) + metricsSystem.registerSource(source) + assert(sources.length === 2) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file From e9ac88754d4c5d58aedd4de8768787300b15eada Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:05:18 +0800 Subject: [PATCH 11/22] Remove twice add Source bug and code clean --- .../src/main/scala/spark/metrics/MetricsSystem.scala | 10 ++++------ .../main/scala/spark/metrics/source/JvmSource.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 6e448cb2a5..bf4487e0fc 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -1,15 +1,15 @@ package spark.metrics -import scala.collection.mutable - import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit +import scala.collection.mutable + import spark.Logging -import spark.metrics.sink._ -import spark.metrics.source._ +import spark.metrics.sink.Sink +import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() @@ -46,7 +46,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() - sources += source.asInstanceOf[Source] registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) @@ -58,7 +57,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { MetricsSystem.DEFAULT_SINKS(kv._1) diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 13270dae3c..8f0870c1a0 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,17 @@ package spark.metrics.source -import java.util.{Map, HashMap => JHashMap} - import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import java.util.{Map, HashMap => JHashMap} + class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet + val gcMetricSet = new GarbageCollectorMetricSet + val memGaugeSet = new MemoryUsageGaugeSet - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) } From 7d2eada451686824bd467641bf1763e82011f2a6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 21:37:21 +0800 Subject: [PATCH 12/22] Add metrics source of DAGScheduler and blockManager Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 13 +++++-- core/src/main/scala/spark/SparkEnv.scala | 6 ++++ .../scala/spark/scheduler/DAGScheduler.scala | 3 +- .../spark/scheduler/DAGSchedulerSource.scala | 31 ++++++++++++++++ .../spark/storage/BlockManagerSource.scala | 35 +++++++++++++++++++ 5 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala create mode 100644 core/src/main/scala/spark/storage/BlockManagerSource.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 24ba605646..a6128a9f30 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -60,11 +60,11 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} @@ -270,6 +270,15 @@ class SparkContext( // Post init taskScheduler.postStartHook() + val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + def initDriverMetrics() = { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + } + + initDriverMetrics() + // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index f2bdc11bdb..d34dafecc5 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -30,6 +30,7 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory +import spark.metrics._ /** @@ -53,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -184,6 +186,9 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + metricsSystem.start() + // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. @@ -213,6 +218,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 9b45fc2938..781e49bdec 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.metrics.MetricsSystem /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,7 +126,6 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) - // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala new file mode 100644 index 0000000000..57aa74512c --- /dev/null +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -0,0 +1,31 @@ +package spark.scheduler + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "DAGScheduler" + + + metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size + }) + + metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size + }) + + metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size + }) + + metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() + }) + + metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size + }) +} diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala new file mode 100644 index 0000000000..c0ce9259c8 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -0,0 +1,35 @@ +package spark.storage + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source +import spark.storage._ + +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "BlockManager" + + metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + maxMem + } + }) + + metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + remainingMem + } + }) + + metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + diskSpaceUsed + } + }) +} From 9cea0c28184c86625f8281eea2af77eff15acb73 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 22:22:29 +0800 Subject: [PATCH 13/22] Refactor metricsSystem unit test, add resource files. --- .../resources/test_metrics_config.properties | 6 ++++ .../resources/test_metrics_system.properties | 7 +++++ .../spark/metrics/MetricsConfigSuite.scala | 30 ++---------------- .../spark/metrics/MetricsSystemSuite.scala | 31 ++----------------- 4 files changed, 19 insertions(+), 55 deletions(-) create mode 100644 core/src/test/resources/test_metrics_config.properties create mode 100644 core/src/test/resources/test_metrics_system.properties diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties new file mode 100644 index 0000000000..2011940003 --- /dev/null +++ b/core/src/test/resources/test_metrics_config.properties @@ -0,0 +1,6 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +*.source.jvm.class = spark.metrics.source.JvmSource +master.sink.console.period = 20 +master.sink.console.unit = minute + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties new file mode 100644 index 0000000000..06afbc6625 --- /dev/null +++ b/core/src/test/resources/test_metrics_system.properties @@ -0,0 +1,7 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +test.sink.console.class = spark.metrics.sink.ConsoleSink +test.sink.dummy.class = spark.metrics.sink.DummySink +test.source.dummy.class = spark.metrics.source.DummySource +test.sink.console.period = 20 +test.sink.console.unit = minute diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index 0c7142c418..f4c83cb644 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -11,24 +11,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val prop = new Properties() - - prop.setProperty("*.sink.console.period", "10") - prop.setProperty("*.sink.console.unit", "second") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") - prop.setProperty("master.sink.console.period", "20") - prop.setProperty("master.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - prop.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { @@ -81,12 +64,5 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 5e8f8fcf80..967be6ec47 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -11,25 +11,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val props = new Properties() - props.setProperty("*.sink.console.period", "10") - props.setProperty("*.sink.console.unit", "second") - props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") - props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") - props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") - props.setProperty("test.sink.console.period", "20") - props.setProperty("test.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - props.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } @@ -54,12 +36,5 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} From 5f8802c1fb106cc04c30b6aca0a6ce98fa5c0e15 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Jul 2013 10:41:43 +0800 Subject: [PATCH 14/22] Register and init metricsSystem in SparkContext Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 10 ++++++++-- core/src/main/scala/spark/SparkEnv.scala | 4 ---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a6128a9f30..1255d0c72e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -67,6 +67,9 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.metrics._ + +import scala.util.DynamicVariable /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -272,9 +275,12 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + metricsSystem.registerSource(dagSchedulerSource) + metricsSystem.registerSource(blockManagerSource) + metricsSystem.start() } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d34dafecc5..125dc55bd8 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,7 +54,6 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -186,8 +185,6 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -218,7 +215,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem, None) } } From 1daff54b2ed92d0bcee7030d7d3ab5c274f80d2f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:28:32 +0800 Subject: [PATCH 15/22] Change Executor MetricsSystem initialize code to SparkEnv --- core/src/main/scala/spark/SparkContext.scala | 6 ++---- core/src/main/scala/spark/SparkEnv.scala | 9 +++++++++ core/src/main/scala/spark/executor/Executor.scala | 6 +++++- .../spark/executor/ExecutorInstrumentation.scala | 10 +++++----- .../scala/spark/executor/MesosExecutorBackend.scala | 10 +--------- .../spark/executor/StandaloneExecutorBackend.scala | 12 ------------ 6 files changed, 22 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1255d0c72e..f1d9d5e442 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -275,12 +275,10 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") def initDriverMetrics() = { - metricsSystem.registerSource(dagSchedulerSource) - metricsSystem.registerSource(blockManagerSource) - metricsSystem.start() + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 125dc55bd8..7b3dc69348 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: metricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -69,6 +70,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + metricsSystem.stop() actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release @@ -185,6 +187,12 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = if (isDriver) { + MetricsSystem.createMetricsSystem("driver") + } else { + MetricsSystem.createMetricsSystem("executor") + } + metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -215,6 +223,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..7179ed84a8 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -86,10 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) + + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) + env.metricsSystem.registerSource(executorInstrumentation) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index 80aadb66b0..ebbcbee742 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,32 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source{ val metricRegistry = new MetricRegistry() val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { - override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 8b6ab0c391..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState -import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -33,9 +32,6 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) - MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -83,17 +79,13 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) { - MesosExecutorBackend.metricsSystem.stop() - } + override def shutdown(d: ExecutorDriver) {} } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 6ef74cd2ff..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -46,8 +45,6 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -55,9 +52,6 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing - - StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) - StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -87,15 +81,9 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } - - override def postStop() { - StandaloneExecutorBackend.metricsSystem.stop() - } } private[spark] object StandaloneExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From a79f6077f0b3bc0110f6e8e15de3068f296f3b81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:29:26 +0800 Subject: [PATCH 16/22] Add Maven metrics library dependency and code changes --- conf/metrics.properties | 4 ++-- core/pom.xml | 8 ++++++++ core/src/main/scala/spark/SparkEnv.scala | 3 ++- pom.xml | 8 ++++++++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/conf/metrics.properties b/conf/metrics.properties index 0bbb6b5229..2ea3aaf345 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,10 +1,10 @@ -# syntax: [prefix].[sink|source].[instance].[options] +# syntax: [instance].[sink|source].[name].[options] *.sink.console.period=10 *.sink.console.unit=second -master.source.jvm.class=spark.metrics.source.JvmSource +*.sink.console.class=spark.metrics.sink.ConsoleSink master.sink.console.period=10 diff --git a/core/pom.xml b/core/pom.xml index 6329b2fbd8..237d988712 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -108,6 +108,14 @@ log4j log4j + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + org.apache.derby diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7b3dc69348..204049d927 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -24,6 +24,7 @@ import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager +import spark.metrics.MetricsSystem import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager @@ -54,7 +55,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: metricsSystem, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort diff --git a/pom.xml b/pom.xml index eb7bd7e9df..3fbd93c7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,14 @@ org.scala-lang scalap ${scala.version} + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 From 5730193e0c8639b654f489c5956e31451d81b7db Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 13:48:09 +0800 Subject: [PATCH 17/22] Fix some typos --- core/src/main/scala/spark/SparkContext.scala | 2 -- core/src/main/scala/spark/SparkEnv.scala | 1 - core/src/main/scala/spark/metrics/sink/JmxSink.scala | 10 +++++----- .../main/scala/spark/metrics/source/JvmSource.scala | 6 ++---- core/src/main/scala/spark/metrics/source/Source.scala | 3 +-- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 7 files changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f1d9d5e442..1e59a4d47d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -69,8 +69,6 @@ import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} import spark.metrics._ -import scala.util.DynamicVariable - /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 204049d927..4a1d341f5d 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -31,7 +31,6 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory -import spark.metrics._ /** diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index f097a631c0..6a40885b78 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,18 +1,18 @@ package spark.metrics.sink -import java.util.Properties - import com.codahale.metrics.{JmxReporter, MetricRegistry} +import java.util.Properties + class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() - + override def start() { reporter.start() } - + override def stop() { reporter.stop() } - + } diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f0870c1a0..79f505079c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -3,15 +3,13 @@ package spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -import java.util.{Map, HashMap => JHashMap} - class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - + val gcMetricSet = new GarbageCollectorMetricSet val memGaugeSet = new MemoryUsageGaugeSet - + metricRegistry.registerAll(gcMetricSet) metricRegistry.registerAll(memGaugeSet) } diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 17cbe2f85a..5607e2c40a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,9 +1,8 @@ package spark.metrics.source -import com.codahale.metrics.MetricSet import com.codahale.metrics.MetricRegistry trait Source { def sourceName: String - def metricRegistry: MetricRegistry + def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 781e49bdec..9b45fc2938 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.metrics.MetricsSystem + /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,6 +126,7 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 825319d3de..d4d70afdd5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -179,7 +179,7 @@ object SparkBuild extends Build { "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( From ed1a3bc206c01974eedd0b1fb1deec183376b5c6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:11:10 +0800 Subject: [PATCH 18/22] continue to refactor code style and functions --- core/src/main/scala/spark/SparkContext.scala | 6 +- .../scala/spark/deploy/master/Master.scala | 6 +- .../deploy/master/MasterInstrumentation.scala | 19 ++-- .../deploy/worker/WorkerInstrumentation.scala | 35 ++++---- .../main/scala/spark/executor/Executor.scala | 8 +- .../executor/ExecutorInstrumentation.scala | 47 +++++----- .../scala/spark/metrics/MetricsConfig.scala | 88 +++++++++---------- .../scala/spark/metrics/MetricsSystem.scala | 40 ++++----- .../spark/metrics/sink/ConsoleSink.scala | 30 +++---- .../scala/spark/metrics/sink/CsvSink.scala | 40 ++++----- .../spark/scheduler/DAGSchedulerSource.scala | 23 +++-- .../spark/storage/BlockManagerSource.scala | 14 +-- .../spark/metrics/MetricsConfigSuite.scala | 32 ++++--- .../spark/metrics/MetricsSystemSuite.scala | 14 +-- 14 files changed, 195 insertions(+), 207 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1e59a4d47d..77cb0ee0cd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -274,9 +274,9 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + def initDriverMetrics() { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index cc0b2d4295..5f67366eb6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,7 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") - + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { @@ -76,7 +76,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - + Master.metricsSystem.registerSource(masterInstrumentation) Master.metricsSystem.start() } @@ -322,7 +322,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - + override def postStop() { Master.metricsSystem.stop() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 61a561c955..4c3708cc4c 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -5,24 +5,21 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class MasterInstrumentation(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "master" - metricRegistry.register(MetricRegistry.name("workers","number"), - new Gauge[Int] { + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { override def getValue: Int = master.workers.size }) - + // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.apps.size + metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { + override def getValue: Int = master.apps.size }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.waitingApps.size + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size }) - } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 94c20a98c1..c76c0b4711 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -8,32 +8,27 @@ private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), - new Gauge[Int] { - override def getValue: Int = worker.executors.size + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + override def getValue: Int = worker.executors.size }) - + // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresUsed + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresUsed }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryUsed + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryUsed }) - + // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresFree + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresFree }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryFree - }) + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryFree + }) } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 7179ed84a8..4ea05dec1c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -69,7 +69,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - + // We may have been called from a shutdown hook. If so, we must not call System.exit(). // (If we do, we will deadlock.) if (!Utils.inShutdown()) { @@ -86,14 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) - + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) env.metricsSystem.registerSource(executorInstrumentation) - + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index ebbcbee742..ad406f41b4 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,27 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" - - // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() - }) - - // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), - new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() - }) - - // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() - }) - - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() - }) -} \ No newline at end of file + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getActiveCount() + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + override def getValue: Long = executor.threadPool.getCompletedTaskCount() + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getPoolSize() + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getMaximumPoolSize() + }) +} diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 7405192058..b1f6a1e596 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,63 +3,63 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable +import scala.collection.mutable.HashMap import scala.util.matching.Regex -private [spark] class MetricsConfig(val configFile: String) { +private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - // Add default properties in case there's no properties file - MetricsConfig.setDefaultProperties(properties) - - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() - } - } - - val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) - if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { - import scala.collection.JavaConversions._ - val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) - } - } - - def getInstance(inst: String) = { - propertyCategories.get(inst) match { - case Some(s) => s - case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) - } - } -} - -private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - - def setDefaultProperties(prop: Properties) { + var propertyCategories: HashMap[String, Properties] = null + + private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") } - - def subProperties(prop: Properties, regex: Regex) = { - val subProperties = new mutable.HashMap[String, Properties] - + + def initilize() { + //Add default properties in case there's no properties file + setDefaultProperties(properties) + + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + } + + propertyCategories = subProperties(properties, INSTANCE_REGEX) + if (propertyCategories.contains(DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + } + + def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { + val subProperties = new HashMap[String, Properties] import scala.collection.JavaConversions._ - prop.foreach { kv => + prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) } } - subProperties } -} \ No newline at end of file + + def getInstance(inst: String): Properties = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(DEFAULT_PREFIX) + } + } +} + diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index bf4487e0fc..54f6c6e4da 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -13,34 +13,37 @@ import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) - + val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - + + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") + + metricsConfig.initilize() registerSources() registerSinks() - + def start() { sinks.foreach(_.start) } - + def stop() { sinks.foreach(_.stop) } - + def registerSource(source: Source) { sources += source registry.register(source.sourceName, source.metricRegistry) } - + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") @@ -52,14 +55,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } } - + def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - + val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + sinkConfigs.foreach { kv => - val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { - MetricsSystem.DEFAULT_SINKS(kv._1) + val classPath = if (DEFAULT_SINKS.contains(kv._1)) { + DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") @@ -76,17 +79,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "millisecond" -> TimeUnit.MILLISECONDS, + "illisecond" -> TimeUnit.MILLISECONDS, "second" -> TimeUnit.SECONDS, "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) - - def createMetricsSystem(instance: String) = new MetricsSystem(instance) + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index d7b7a9e501..c67c0ee912 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,34 +8,34 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) } - + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object ConsoleSink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" - - val CONSOLE_KEY_PERIOD = "period" - val CONSOLE_KEY_UNIT = "unit" -} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index e6c5bffd3c..a8ca819e87 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -9,21 +9,29 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_DIR = "/tmp/" + + val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) } - - val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + + val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s - case None => CsvSink.CSV_DEFAULT_DIR + case None => CSV_DEFAULT_DIR } - + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -31,21 +39,11 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .build(new File(pollDir)) override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object CsvSink { - val CSV_KEY_PERIOD = "period" - val CSV_KEY_UNIT = "unit" - val CSV_KEY_DIR = "directory" - - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" - val CSV_DEFAULT_DIR = "/tmp/" -} - diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 57aa74512c..38158b8a2b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -5,27 +5,26 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - - metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.nextRunId.get() + metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.activeJobs.size + metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index c0ce9259c8..f964827102 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -6,29 +6,29 @@ import spark.metrics.source.Source import spark.storage._ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) maxMem } }) - metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) remainingMem } }) - metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) diskSpaceUsed } }) diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index f4c83cb644..bb1be4f4fc 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -9,60 +9,64 @@ import spark.metrics._ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { val conf = new MetricsConfig("dummy-file") + conf.initilize() + assert(conf.properties.size() === 2) assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") assert(conf.properties.getProperty("test-for-dummy") === null) - + val property = conf.getInstance("random") assert(property.size() === 2) assert(property.getProperty("sink.jmx.enabled") === "default") assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } - + test("MetricsConfig with properties set") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val masterProp = conf.getInstance("master") assert(masterProp.size() === 4) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minute") assert(masterProp.getProperty("sink.jmx.enabled") === "default") assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") - + val workerProp = conf.getInstance("worker") assert(workerProp.size() === 4) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "second") } - + test("MetricsConfig with subProperties") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val propCategories = conf.propertyCategories assert(propCategories.size === 2) - + val masterProp = conf.getInstance("master") - val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) assert(sourceProps.size === 1) assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") - - val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + + val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) assert(sinkProps.size === 2) assert(sinkProps.contains("console")) assert(sinkProps.contains("jmx")) - + val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - + val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } + } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 967be6ec47..f29bb9db67 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -9,32 +9,32 @@ import spark.metrics._ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } - + test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 1) assert(sources(0).sourceName === "jvm") } - + test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 2) - + val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } + } } From 5b4a2f2094258920613f50c0d337e1e92c08d012 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:31:53 +0800 Subject: [PATCH 19/22] Add metrics config template file --- conf/metrics.properties | 15 --------------- conf/metrics.properties.template | 14 ++++++++++++++ 2 files changed, 14 insertions(+), 15 deletions(-) delete mode 100644 conf/metrics.properties create mode 100644 conf/metrics.properties.template diff --git a/conf/metrics.properties b/conf/metrics.properties deleted file mode 100644 index 2ea3aaf345..0000000000 --- a/conf/metrics.properties +++ /dev/null @@ -1,15 +0,0 @@ -# syntax: [instance].[sink|source].[name].[options] - -*.sink.console.period=10 - -*.sink.console.unit=second - -*.sink.console.class=spark.metrics.sink.ConsoleSink - -master.sink.console.period=10 - -master.sink.console.unit=second - -worker.sink.console.period=20 - -worker.sink.console.unit=second diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template new file mode 100644 index 0000000000..b4204e1deb --- /dev/null +++ b/conf/metrics.properties.template @@ -0,0 +1,14 @@ +# syntax: [instance].[sink|source].[name].[options] + +#*.sink.console.period=10 + +#*.sink.console.unit=second + +#master.source.jvm.class=spark.metrics.source.JvmSource + +#worker.source.jvm.class=spark.metrics.source.JvmSource + +#driver.source.jvm.class=spark.metrics.source.JvmSource + +#executor.source.jvm.class=spark.metrics.source.JvmSource + From 05637de8423da85c5934cdfa8f07254133a58474 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 21:16:39 +0800 Subject: [PATCH 20/22] Change class xxxInstrumentation to class xxxSource --- core/src/main/scala/spark/deploy/master/Master.scala | 4 ++-- .../{MasterInstrumentation.scala => MasterSource.scala} | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 4 ++-- .../{WorkerInstrumentation.scala => WorkerSource.scala} | 2 +- core/src/main/scala/spark/executor/Executor.scala | 4 ++-- .../{ExecutorInstrumentation.scala => ExecutorSource.scala} | 2 +- core/src/test/scala/spark/metrics/MetricsSystemSuite.scala | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/scala/spark/deploy/master/{MasterInstrumentation.scala => MasterSource.scala} (90%) rename core/src/main/scala/spark/deploy/worker/{WorkerInstrumentation.scala => WorkerSource.scala} (93%) rename core/src/main/scala/spark/executor/{ExecutorInstrumentation.scala => ExecutorSource.scala} (94%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5f67366eb6..1d592206c0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,7 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterInstrumentation = new MasterInstrumentation(this) + val masterSource = new MasterSource(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -77,7 +77,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.registerSource(masterSource) Master.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala rename to core/src/main/scala/spark/deploy/master/MasterSource.scala index 4c3708cc4c..f94e5b2c34 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -private[spark] class MasterInstrumentation(val master: Master) extends Source { +private[spark] class MasterSource(val master: Master) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "master" diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index eaa1c1806f..5c0f77fd75 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,7 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val workerInstrumentation = new WorkerInstrumentation(this) + val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -102,7 +102,7 @@ private[spark] class Worker( connectToMaster() startWebUi() - Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.registerSource(workerSource) Worker.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala similarity index 93% rename from core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala rename to core/src/main/scala/spark/deploy/worker/WorkerSource.scala index c76c0b4711..539eac71bd 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { +private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 4ea05dec1c..8a74a8d853 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -87,12 +87,12 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } ) - val executorInstrumentation = new ExecutorInstrumentation(this) + val executorSource = new ExecutorSource(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) - env.metricsSystem.registerSource(executorInstrumentation) + env.metricsSystem.registerSource(executorSource) private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala similarity index 94% rename from core/src/main/scala/spark/executor/ExecutorInstrumentation.scala rename to core/src/main/scala/spark/executor/ExecutorSource.scala index ad406f41b4..d8b531cb58 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source { +class ExecutorSource(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index f29bb9db67..462c28e894 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -33,7 +33,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 1) assert(sinks.length === 2) - val source = new spark.deploy.master.MasterInstrumentation(null) + val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) assert(sources.length === 2) } From 8d1ef7f2dfc453137b8dbbb72a7f1ca93b57bb15 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 4 Jul 2013 10:11:30 +0800 Subject: [PATCH 21/22] Code style changes --- conf/metrics.properties.template | 2 ++ .../src/main/scala/spark/deploy/master/MasterSource.scala | 4 ++-- core/src/main/scala/spark/metrics/MetricsConfig.scala | 8 ++++---- core/src/main/scala/spark/metrics/sink/ConsoleSink.scala | 1 - 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index b4204e1deb..c7e24aa36c 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,5 +1,7 @@ # syntax: [instance].[sink|source].[name].[options] +#*.sink.console.class=spark.metrics.sink.ConsoleSink + #*.sink.console.period=10 #*.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index f94e5b2c34..65c22320d6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -1,6 +1,6 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge,MetricRegistry} +import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source @@ -10,7 +10,7 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { - override def getValue: Int = master.workers.size + override def getValue: Int = master.workers.size }) // Gauge for application numbers in cluster diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index b1f6a1e596..5066b7ac22 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,14 +3,14 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable.HashMap +import scala.collection.mutable import scala.util.matching.Regex private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - var propertyCategories: HashMap[String, Properties] = null + var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") @@ -43,8 +43,8 @@ private[spark] class MetricsConfig(val configFile: String) { } } - def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { - val subProperties = new HashMap[String, Properties] + def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { + val subProperties = new mutable.HashMap[String, Properties] import scala.collection.JavaConversions._ prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index c67c0ee912..437f24a575 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,7 +8,6 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" val CONSOLE_DEFAULT_UNIT = "second" From 31ec72b243459e0d1d9c89033328e1d79f3aa76f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Jul 2013 10:57:19 +0800 Subject: [PATCH 22/22] Code refactor according to comments --- conf/metrics.properties.template | 73 ++++++++++++++++- .../scala/spark/deploy/master/Master.scala | 11 +-- .../spark/deploy/master/MasterSource.scala | 2 +- .../scala/spark/deploy/worker/Worker.scala | 11 +-- .../spark/deploy/worker/WorkerSource.scala | 12 +-- .../scala/spark/executor/ExecutorSource.scala | 8 +- .../scala/spark/metrics/MetricsConfig.scala | 50 +++++++----- .../scala/spark/metrics/MetricsSystem.scala | 81 ++++++++++++++----- .../spark/metrics/sink/ConsoleSink.scala | 12 +-- .../scala/spark/metrics/sink/CsvSink.scala | 12 +-- .../spark/scheduler/DAGSchedulerSource.scala | 10 +-- .../spark/storage/BlockManagerSource.scala | 31 ++++--- .../resources/test_metrics_config.properties | 6 +- .../resources/test_metrics_system.properties | 4 +- .../spark/metrics/MetricsConfigSuite.scala | 38 ++++----- .../spark/metrics/MetricsSystemSuite.scala | 13 ++- 16 files changed, 249 insertions(+), 125 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index c7e24aa36c..0486ca4c79 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,11 +1,82 @@ # syntax: [instance].[sink|source].[name].[options] +# "instance" specify "who" (the role) use metrics system. In spark there are +# several roles like master, worker, executor, driver, these roles will +# create metrics system for monitoring. So instance represents these roles. +# Currently in Spark, several instances have already implemented: master, +# worker, executor, driver. +# +# [instance] field can be "master", "worker", "executor", "driver", which means +# only the specified instance has this property. +# a wild card "*" can be used to represent instance name, which means all the +# instances will have this property. +# +# "source" specify "where" (source) to collect metrics data. In metrics system, +# there exists two kinds of source: +# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will +# collect Spark component's internal state, these sources are related to +# instance and will be added after specific metrics system is created. +# 2. Common source, like JvmSource, which will collect low level state, is +# configured by configuration and loaded through reflection. +# +# "sink" specify "where" (destination) to output metrics data to. Several sinks +# can be coexisted and flush metrics to all these sinks. +# +# [sink|source] field specify this property is source related or sink, this +# field can only be source or sink. +# +# [name] field specify the name of source or sink, this is custom defined. +# +# [options] field is the specific property of this source or sink, this source +# or sink is responsible for parsing this property. +# +# Notes: +# 1. Sinks should be added through configuration, like console sink, class +# full name should be specified by class property. +# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, +# it should be attention minimal polling period is 1 seconds, any period +# below than 1s is illegal. +# 3. Wild card property can be overlapped by specific instance property, for +# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 4. A metrics specific configuration +# "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be +# added to Java property using -Dspark.metrics.conf=xxx if you want to +# customize metrics system, or you can put it in ${SPARK_HOME}/conf, +# metrics system will search and load it automatically. + +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=spark.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name #*.sink.console.class=spark.metrics.sink.ConsoleSink +# Polling period for ConsoleSink #*.sink.console.period=10 -#*.sink.console.unit=second +#*.sink.console.unit=seconds +# Master instance overlap polling period +#master.sink.console.period=15 + +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances +#*.sink.csv.class=spark.metrics.sink.CsvSink + +# Polling period for CsvSink +#*.sink.csv.period=1 + +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Worker instance overlap polling period +#worker.sink.csv.period=10 + +#worker.sink.csv.unit=minutes + +# Enable jvm source for instance master, worker, driver and executor #master.source.jvm.class=spark.metrics.source.JvmSource #worker.source.jvm.class=spark.metrics.source.JvmSource diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 1d592206c0..9692af5295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,6 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") + val metricsSystem = MetricsSystem.createMetricsSystem("master") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -77,12 +78,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterSource) - Master.metricsSystem.start() + metricsSystem.registerSource(masterSource) + metricsSystem.start() } override def postStop() { webUi.stop() + metricsSystem.stop() } override def receive = { @@ -322,17 +324,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - - override def postStop() { - Master.metricsSystem.stop() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index 65c22320d6..b8cfa6a773 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -19,7 +19,7 @@ private[spark] class MasterSource(val master: Master) extends Source { }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("waitingApps", "number"), new Gauge[Int] { override def getValue: Int = master.waitingApps.size }) } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 5c0f77fd75..8fa0d12b82 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,6 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val metricsSystem = MetricsSystem.createMetricsSystem("worker") val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -100,10 +101,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() - startWebUi() - Worker.metricsSystem.registerSource(workerSource) - Worker.metricsSystem.start() + metricsSystem.registerSource(workerSource) + metricsSystem.start() } def connectToMaster() { @@ -185,14 +185,11 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() - - Worker.metricsSystem.stop() + metricsSystem.stop() } } private[spark] object Worker { - private val metricsSystem = MetricsSystem.createMetricsSystem("worker") - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala index 539eac71bd..39cb8e5690 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -8,27 +8,27 @@ private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("executors", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresUsed", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("memUsed", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresFree", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + // Gauge for memory free of this worker + metricRegistry.register(MetricRegistry.name("memFree", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) } diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala index d8b531cb58..94116edfcf 100644 --- a/core/src/main/scala/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -9,22 +9,22 @@ class ExecutorSource(val executor: Executor) extends Source { val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "activeTask", "count"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("threadpool", "completeTask", "count"), new Gauge[Long] { override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "currentPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "maxPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 5066b7ac22..ed505b0aa7 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,44 +1,58 @@ package spark.metrics import java.util.Properties -import java.io.{File, FileInputStream} +import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex -private[spark] class MetricsConfig(val configFile: String) { - val properties = new Properties() +import spark.Logging + +private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { + initLogging() + val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + val METRICS_CONF = "metrics.properties" + + val properties = new Properties() var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { - prop.setProperty("*.sink.jmx.enabled", "default") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + // empty function, any default property can be set here } - def initilize() { + def initialize() { //Add default properties in case there's no properties file setDefaultProperties(properties) - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + // If spark.metrics.conf is not set, try to get file in class path + var is: InputStream = null + try { + is = configFile match { + case Some(f) => new FileInputStream(f) + case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) } + + if (is != null) { + properties.load(is) + } + } catch { + case e: Exception => logError("Error loading configure file", e) + } finally { + if (is != null) is.close() } propertyCategories = subProperties(properties, INSTANCE_REGEX) if (propertyCategories.contains(DEFAULT_PREFIX)) { import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) + for { (inst, prop) <- propertyCategories + if (inst != DEFAULT_PREFIX) + (k, v) <- defaultProperty + if (prop.getProperty(k) == null) } { + prop.setProperty(k, v) } } } @@ -58,7 +72,7 @@ private[spark] class MetricsConfig(val configFile: String) { def getInstance(inst: String): Properties = { propertyCategories.get(inst) match { case Some(s) => s - case None => propertyCategories(DEFAULT_PREFIX) + case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) } } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 54f6c6e4da..2f87577ff3 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -11,19 +11,51 @@ import spark.Logging import spark.metrics.sink.Sink import spark.metrics.source.Source +/** + * Spark Metrics System, created by specific "instance", combined by source, + * sink, periodically poll source metrics data to sink destinations. + * + * "instance" specify "who" (the role) use metrics system. In spark there are several roles + * like master, worker, executor, client driver, these roles will create metrics system + * for monitoring. So instance represents these roles. Currently in Spark, several instances + * have already implemented: master, worker, executor, driver. + * + * "source" specify "where" (source) to collect metrics data. In metrics system, there exists + * two kinds of source: + * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect + * Spark component's internal state, these sources are related to instance and will be + * added after specific metrics system is created. + * 2. Common source, like JvmSource, which will collect low level state, is configured by + * configuration and loaded through reflection. + * + * "sink" specify "where" (destination) to output metrics data to. Several sinks can be + * coexisted and flush metrics to all these sinks. + * + * Metrics configuration format is like below: + * [instance].[sink|source].[name].[options] = xxxx + * + * [instance] can be "master", "worker", "executor", "driver", which means only the specified + * instance has this property. + * wild card "*" can be used to replace instance name, which means all the instances will have + * this property. + * + * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * + * [name] specify the name of sink or source, it is custom defined. + * + * [options] is the specific property of this source or sink. + */ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") - val metricsConfig = new MetricsConfig(confFile) + val confFile = System.getProperty("spark.metrics.conf") + val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - - metricsConfig.initilize() + metricsConfig.initialize() registerSources() registerSinks() @@ -37,7 +69,11 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName, source.metricRegistry) + try { + registry.register(source.sourceName, source.metricRegistry) + } catch { + case e: IllegalArgumentException => logInfo("Metrics already registered", e) + } } def registerSources() { @@ -51,7 +87,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) } } } @@ -61,18 +97,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) sinkConfigs.foreach { kv => - val classPath = if (DEFAULT_SINKS.contains(kv._1)) { - DEFAULT_SINKS(kv._1) - } else { - // For non-default sink, a property class should be set and create using reflection - kv._2.getProperty("class") - } + val classPath = kv._2.getProperty("class") try { - val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry]) .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } @@ -81,12 +113,17 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "illisecond" -> TimeUnit.MILLISECONDS, - "second" -> TimeUnit.SECONDS, - "minute" -> TimeUnit.MINUTES, - "hour" -> TimeUnit.HOURS, - "day" -> TimeUnit.DAYS) - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + val MINIMAL_POLL_PERIOD = 1 + + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } + } + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 437f24a575..eaaac5d153 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,22 +8,24 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" + val CONSOLE_DEFAULT_PERIOD = 10 + val CONSOLE_DEFAULT_UNIT = "SECONDS" val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index a8ca819e87..aa5bff0d34 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -13,19 +13,21 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_PERIOD = 10 + val CSV_DEFAULT_UNIT = "SECONDS" val CSV_DEFAULT_DIR = "/tmp/" val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 38158b8a2b..87d27cc70d 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -8,23 +8,23 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "failedStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "runningStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "waitingStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index f964827102..4faa715c94 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -9,27 +9,40 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - maxMem + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + maxMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - remainingMem + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + remainingMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "memUsed", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) - diskSpaceUsed + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + (maxMem - remainingMem) / 1024 / 1024 + } + }) + + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed", "MBytes"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) + + diskSpaceUsed / 1024 / 1024 } }) } diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 2011940003..2b31ddf2eb 100644 --- a/core/src/test/resources/test_metrics_config.properties +++ b/core/src/test/resources/test_metrics_config.properties @@ -1,6 +1,6 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds *.source.jvm.class = spark.metrics.source.JvmSource master.sink.console.period = 20 -master.sink.console.unit = minute - +master.sink.console.unit = minutes + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 06afbc6625..d5479f0298 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -1,7 +1,7 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds test.sink.console.class = spark.metrics.sink.ConsoleSink test.sink.dummy.class = spark.metrics.sink.DummySink test.source.dummy.class = spark.metrics.source.DummySource test.sink.console.period = 20 -test.sink.console.unit = minute +test.sink.console.unit = minutes diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index bb1be4f4fc..87cd2ffad2 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -15,40 +15,36 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig("dummy-file") - conf.initilize() + val conf = new MetricsConfig(Option("dummy-file")) + conf.initialize() - assert(conf.properties.size() === 2) - assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") - assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.size() === 0) assert(conf.properties.getProperty("test-for-dummy") === null) val property = conf.getInstance("random") - assert(property.size() === 2) - assert(property.getProperty("sink.jmx.enabled") === "default") - assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + assert(property.size() === 0) } test("MetricsConfig with properties set") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val masterProp = conf.getInstance("master") - assert(masterProp.size() === 4) + assert(masterProp.size() === 3) assert(masterProp.getProperty("sink.console.period") === "20") - assert(masterProp.getProperty("sink.console.unit") === "minute") - assert(masterProp.getProperty("sink.jmx.enabled") === "default") - assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.console.unit") === "minutes") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") val workerProp = conf.getInstance("worker") - assert(workerProp.size() === 4) + assert(workerProp.size() === 3) assert(workerProp.getProperty("sink.console.period") === "10") - assert(workerProp.getProperty("sink.console.unit") === "second") + assert(workerProp.getProperty("sink.console.unit") === "seconds") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } test("MetricsConfig with subProperties") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val propCategories = conf.propertyCategories assert(propCategories.size === 2) @@ -59,14 +55,10 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) - assert(sinkProps.size === 2) + assert(sinkProps.size === 1) assert(sinkProps.contains("console")) - assert(sinkProps.contains("jmx")) val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - - val jmxProps = sinkProps("jmx") - assert(jmxProps.size() === 1) } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 462c28e894..c189996417 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -12,7 +12,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() - System.setProperty("spark.metrics.conf.file", filePath) + System.setProperty("spark.metrics.conf", filePath) } test("MetricsSystem with default config") { @@ -20,9 +20,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 1) - assert(sources(0).sourceName === "jvm") + assert(sources.length === 0) + assert(sinks.length === 0) } test("MetricsSystem with sources add") { @@ -30,11 +29,11 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 2) + assert(sources.length === 0) + assert(sinks.length === 1) val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 2) + assert(sources.length === 1) } }