Skip to content
Snippets Groups Projects
Commit 03f98711 authored by jerryshao's avatar jerryshao
Browse files

MetricsSystem refactor

parent c3daad3f
No related branches found
No related tags found
No related merge requests found
Showing
with 193 additions and 149 deletions
...@@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} ...@@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import spark.deploy._ import spark.deploy._
import spark.{Logging, SparkException, Utils} import spark.{Logging, SparkException, Utils}
import spark.metrics.MetricsSystem
import spark.util.AkkaUtils import spark.util.AkkaUtils
import ui.MasterWebUI import ui.MasterWebUI
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
with Logging with MasterInstrumentation {
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
...@@ -57,6 +57,8 @@ with Logging with MasterInstrumentation { ...@@ -57,6 +57,8 @@ with Logging with MasterInstrumentation {
val webUi = new MasterWebUI(self, webUiPort) val webUi = new MasterWebUI(self, webUiPort)
Utils.checkHost(host, "Expected hostname") Utils.checkHost(host, "Expected hostname")
val masterInstrumentation = new MasterInstrumentation(this)
val masterPublicAddress = { val masterPublicAddress = {
val envVar = System.getenv("SPARK_PUBLIC_DNS") val envVar = System.getenv("SPARK_PUBLIC_DNS")
...@@ -75,7 +77,7 @@ with Logging with MasterInstrumentation { ...@@ -75,7 +77,7 @@ with Logging with MasterInstrumentation {
webUi.start() webUi.start()
context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers())
initialize(this) Master.metricsSystem.registerSource(masterInstrumentation)
} }
override def postStop() { override def postStop() {
...@@ -319,21 +321,22 @@ with Logging with MasterInstrumentation { ...@@ -319,21 +321,22 @@ with Logging with MasterInstrumentation {
removeWorker(worker) removeWorker(worker)
} }
} }
override def postStop() {
uninitialize()
}
} }
private[spark] object Master { private[spark] object Master {
private val systemName = "sparkMaster" private val systemName = "sparkMaster"
private val actorName = "Master" private val actorName = "Master"
private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
private val metricsSystem = MetricsSystem.createMetricsSystem("master")
def main(argStrings: Array[String]) { def main(argStrings: Array[String]) {
val args = new MasterArguments(argStrings) val args = new MasterArguments(argStrings)
val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
metricsSystem.start()
actorSystem.awaitTermination() actorSystem.awaitTermination()
metricsSystem.stop()
} }
/** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
......
package spark.deploy.master 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 { import spark.metrics.source.Source
var masterInst: Option[Master] = None
val metricRegistry = new MetricRegistry() private[spark] class MasterInstrumentation(val master: Master) extends Source {
val className = classOf[Master].getName()
override def registryHandler = metricRegistry val instrumentationName = "master"
override def instance = "master" override def sourceName = instrumentationName
def initialize(master: Master) { override def getMetrics(): Map[String, Metric] = {
masterInst = Some(master) val gauges = new JHashMap[String, Metric]
// Register all the sources // Gauge for worker numbers in cluster
registerSources() gauges.put(className + ".workers.number", new Gauge[Int] {
override def getValue: Int = master.workers.size
})
// Register and start all the sinks // Gauge for application numbers in cluster
registerSinks() gauges.put(className + ".apps.number", new Gauge[Int] {
} override def getValue: Int = master.apps.size
})
def uninitialize() {
unregisterSinks() // Gauge for waiting application numbers in cluster
gauges.put(className + ".waiting_apps.number", new Gauge[Int] {
override def getValue: Int = master.waitingApps.size
})
gauges
} }
// 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
...@@ -23,6 +23,7 @@ import akka.util.duration._ ...@@ -23,6 +23,7 @@ import akka.util.duration._
import spark.{Logging, Utils} import spark.{Logging, Utils}
import spark.util.AkkaUtils import spark.util.AkkaUtils
import spark.deploy._ import spark.deploy._
import spark.metrics.MetricsSystem
import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
...@@ -41,7 +42,7 @@ private[spark] class Worker( ...@@ -41,7 +42,7 @@ private[spark] class Worker(
memory: Int, memory: Int,
masterUrl: String, masterUrl: String,
workDirPath: String = null) workDirPath: String = null)
extends Actor with Logging with WorkerInstrumentation { extends Actor with Logging {
Utils.checkHost(host, "Expected hostname") Utils.checkHost(host, "Expected hostname")
assert (port > 0) assert (port > 0)
...@@ -67,6 +68,8 @@ private[spark] class Worker( ...@@ -67,6 +68,8 @@ private[spark] class Worker(
var coresUsed = 0 var coresUsed = 0
var memoryUsed = 0 var memoryUsed = 0
val workerInstrumentation = new WorkerInstrumentation(this)
def coresFree: Int = cores - coresUsed def coresFree: Int = cores - coresUsed
def memoryFree: Int = memory - memoryUsed def memoryFree: Int = memory - memoryUsed
...@@ -99,7 +102,8 @@ private[spark] class Worker( ...@@ -99,7 +102,8 @@ private[spark] class Worker(
connectToMaster() connectToMaster()
startWebUi() startWebUi()
initialize(this) Worker.metricsSystem.registerSource(workerInstrumentation)
Worker.metricsSystem.start()
} }
def connectToMaster() { def connectToMaster() {
...@@ -182,11 +186,13 @@ private[spark] class Worker( ...@@ -182,11 +186,13 @@ private[spark] class Worker(
executors.values.foreach(_.kill()) executors.values.foreach(_.kill())
webUi.stop() webUi.stop()
uninitialize() Worker.metricsSystem.stop()
} }
} }
private[spark] object Worker { private[spark] object Worker {
private val metricsSystem = MetricsSystem.createMetricsSystem("worker")
def main(argStrings: Array[String]) { def main(argStrings: Array[String]) {
val args = new WorkerArguments(argStrings) val args = new WorkerArguments(argStrings)
val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
......
package spark.deploy.worker 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 { import spark.metrics.source.Source
var workerInst: Option[Worker] = None
val metricRegistry = new MetricRegistry() 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) { override def sourceName = "worker"
workerInst = Some(worker)
// Register all the sources override def getMetrics: Map[String, Metric] = {
registerSources() val gauges = new JHashMap[String, Metric]
// Register and start all the sinks // Gauge for executors number
registerSinks() gauges.put(className + ".executor.number", new Gauge[Int]{
} override def getValue: Int = worker.executors.size
})
def uninitialize() {
unregisterSinks() 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
} }
}
// Gauge for executors number //private[spark] trait WorkerInstrumentation extends AbstractInstrumentation {
metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), // var workerInst: Option[Worker] = None
new Gauge[Int] { // val metricRegistry = new MetricRegistry()
override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) //
}) // override def registryHandler = metricRegistry
//
// Gauge for cores used of this worker // override def instance = "worker"
metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), //
new Gauge[Int] { // def initialize(worker: Worker) {
override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) // workerInst = Some(worker)
}) //
// registerSources()
// Gauge for memory used of this worker // registerSinks()
metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), // }
new Gauge[Int] { //
override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) // def uninitialize() {
}) // unregisterSinks()
// }
// Gauge for cores free of this worker //
metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), // // Gauge for executors number
new Gauge[Int] { // metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"),
override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) // new Gauge[Int] {
}) // override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0)
// })
// Gauge for memory used of this worker //
metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), // // Gauge for cores used of this worker
new Gauge[Int] { // metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"),
override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) // new Gauge[Int] {
}) // override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0)
} // })
\ No newline at end of file //
// // 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
...@@ -2,7 +2,7 @@ package spark.metrics ...@@ -2,7 +2,7 @@ package spark.metrics
import scala.collection.mutable import scala.collection.mutable
import com.codahale.metrics.{JmxReporter, MetricRegistry} import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry}
import java.util.Properties import java.util.Properties
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
...@@ -11,70 +11,76 @@ import spark.Logging ...@@ -11,70 +11,76 @@ import spark.Logging
import spark.metrics.sink._ import spark.metrics.sink._
import spark.metrics.source._ import spark.metrics.source._
private [spark] trait AbstractInstrumentation extends Logging { private[spark] class MetricsSystem private (val instance: String) extends Logging {
initLogging() 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 confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE)
val metricsConfig = new MetricsConfig(confFile) val metricsConfig = new MetricsConfig(confFile)
val sinks = new mutable.ArrayBuffer[Sink] val sinks = new mutable.ArrayBuffer[Sink]
val sources = new mutable.ArrayBuffer[Source] 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() { def registerSources() {
val instConfig = metricsConfig.getInstance(instance) 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 => sourceConfigs.foreach { kv =>
val classPath = kv._2.getProperty("class") val classPath = kv._2.getProperty("class")
try { try {
val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) val source = Class.forName(classPath).newInstance()
.newInstance(registryHandler)
sources += source.asInstanceOf[Source] sources += source.asInstanceOf[Source]
MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet])
} catch { } catch {
case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) case e: Exception => logError("source class " + classPath + " cannot be instantialized", e)
} }
} }
sources.foreach(_.registerSource)
} }
def registerSinks() { def registerSinks() {
val instConfig = metricsConfig.getInstance(instance) 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 // Register JMX sink as a default sink
sinks += new JmxSink(registryHandler) sinks += new JmxSink(MetricsSystem.registry)
// Register other sinks according to conf // Register other sinks according to conf
sinkConfigs.foreach { kv => sinkConfigs.foreach { kv =>
val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) {
AbstractInstrumentation.DEFAULT_SINKS(kv._1) MetricsSystem.DEFAULT_SINKS(kv._1)
} else { } else {
// For non-default sink, a property class should be set and create using reflection // For non-default sink, a property class should be set and create using reflection
kv._2.getProperty("class") kv._2.getProperty("class")
} }
try { try {
val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry])
.newInstance(kv._2, registryHandler) .newInstance(kv._2, MetricsSystem.registry)
sinks += sink.asInstanceOf[Sink] sinks += sink.asInstanceOf[Sink]
} catch { } catch {
case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e)
} }
} }
sinks.foreach(_.registerSink) sinks.foreach(_.start)
}
def unregisterSinks() {
sinks.foreach(_.unregisterSink)
} }
} }
object AbstractInstrumentation { private[spark] object MetricsSystem {
val registry = new MetricRegistry()
val DEFAULT_SINKS = Map( val DEFAULT_SINKS = Map(
"console" -> "spark.metrics.sink.ConsoleSink", "console" -> "spark.metrics.sink.ConsoleSink",
"csv" -> "spark.metrics.sink.CsvSink") "csv" -> "spark.metrics.sink.CsvSink")
...@@ -88,4 +94,6 @@ object AbstractInstrumentation { ...@@ -88,4 +94,6 @@ object AbstractInstrumentation {
"minute" -> TimeUnit.MINUTES, "minute" -> TimeUnit.MINUTES,
"hour" -> TimeUnit.HOURS, "hour" -> TimeUnit.HOURS,
"day" -> TimeUnit.DAYS) "day" -> TimeUnit.DAYS)
def createMetricsSystem(instance: String) = new MetricsSystem(instance)
} }
\ No newline at end of file
...@@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit ...@@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit
import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import com.codahale.metrics.{ConsoleReporter, MetricRegistry}
import spark.metrics.AbstractInstrumentation import spark.metrics.MetricsSystem
class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink {
val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match {
...@@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend ...@@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend
} }
val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match {
case Some(s) => AbstractInstrumentation.timeUnits(s) case Some(s) => MetricsSystem.timeUnits(s)
case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT)
} }
var reporter: ConsoleReporter = _ var reporter: ConsoleReporter = _
override def registerSink() { override def start() {
reporter = ConsoleReporter.forRegistry(registry) reporter = ConsoleReporter.forRegistry(registry)
.convertDurationsTo(TimeUnit.MILLISECONDS) .convertDurationsTo(TimeUnit.MILLISECONDS)
.convertRatesTo(TimeUnit.SECONDS) .convertRatesTo(TimeUnit.SECONDS)
...@@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend ...@@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend
reporter.start(pollPeriod, pollUnit) reporter.start(pollPeriod, pollUnit)
} }
override def unregisterSink() { override def stop() {
reporter.stop() reporter.stop()
} }
} }
......
...@@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit ...@@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit
import com.codahale.metrics.{CsvReporter, MetricRegistry} import com.codahale.metrics.{CsvReporter, MetricRegistry}
import spark.metrics.AbstractInstrumentation import spark.metrics.MetricsSystem
class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink {
val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match {
...@@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si ...@@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si
} }
val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match {
case Some(s) => AbstractInstrumentation.timeUnits(s) case Some(s) => MetricsSystem.timeUnits(s)
case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT)
} }
val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match {
...@@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si ...@@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si
var reporter: CsvReporter = _ var reporter: CsvReporter = _
override def registerSink() { override def start() {
reporter = CsvReporter.forRegistry(registry) reporter = CsvReporter.forRegistry(registry)
.formatFor(Locale.US) .formatFor(Locale.US)
.convertDurationsTo(TimeUnit.MILLISECONDS) .convertDurationsTo(TimeUnit.MILLISECONDS)
...@@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si ...@@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si
reporter.start(pollPeriod, pollUnit) reporter.start(pollPeriod, pollUnit)
} }
override def unregisterSink() { override def stop() {
reporter.stop() reporter.stop()
} }
} }
......
...@@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry} ...@@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry}
class JmxSink(registry: MetricRegistry) extends Sink { class JmxSink(registry: MetricRegistry) extends Sink {
var reporter: JmxReporter = _ var reporter: JmxReporter = _
override def registerSink() { override def start() {
reporter = JmxReporter.forRegistry(registry).build() reporter = JmxReporter.forRegistry(registry).build()
reporter.start() reporter.start()
} }
override def unregisterSink() { override def stop() {
reporter.stop() reporter.stop()
} }
......
package spark.metrics.sink package spark.metrics.sink
trait Sink { trait Sink {
def registerSink: Unit def start: Unit
def unregisterSink: Unit def stop: Unit
} }
\ No newline at end of file
package spark.metrics.source package spark.metrics.source
import com.codahale.metrics.MetricRegistry import java.util.{Map, HashMap => JHashMap}
import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet}
class JvmSource(registry: MetricRegistry) extends Source { import com.codahale.metrics.Metric
// Initialize memory usage gauge for jvm import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet}
val memUsageMetricSet = new MemoryUsageGaugeSet
class JvmSource extends Source {
// Initialize garbage collection usage gauge for jvm override def sourceName = "jvm"
val gcMetricSet = new GarbageCollectorMetricSet
override def getMetrics(): Map[String, Metric] = {
override def registerSource() { val gauges = new JHashMap[String, Metric]
registry.registerAll(memUsageMetricSet)
registry.registerAll(gcMetricSet) 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
package spark.metrics.source package spark.metrics.source
trait Source { import com.codahale.metrics.MetricSet
def registerSource: Unit
trait Source extends MetricSet {
def sourceName: String
} }
\ No newline at end of file
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment