From 82679744ea1be5cee0b52741f72c2d836e8bb7b0 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 9 Nov 2015 16:32:33 +0000 Subject: [PATCH 01/22] [SPARK-11373] First pass at adding metrics to the history server, with the FsHistoryProvider counting 1. Number of update attempts and number of failed updates 2. Time for updates and app UI loads The HistoryServer sets up the codahale metrics for the Web under metrics/ with metrics/metrics behind metrics, metrics/health any health probes and metrics/threads a thread dump. --- core/pom.xml | 8 +++ .../history/ApplicationHistoryProvider.scala | 14 +++++ .../deploy/history/FsHistoryProvider.scala | 60 +++++++++++++++++++ .../spark/deploy/history/HistoryServer.scala | 40 +++++++++++++ .../apache/spark/metrics/MetricsSystem.scala | 4 +- .../history/FsHistoryProviderSuite.scala | 27 +++++++-- pom.xml | 10 ++++ 7 files changed, 156 insertions(+), 7 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 97a463abbefdd..1eaec171c93d5 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -253,6 +253,14 @@ io.dropwizard.metrics metrics-graphite + + io.dropwizard.metrics + metrics-healthchecks + + + io.dropwizard.metrics + metrics-servlets + com.fasterxml.jackson.core jackson-databind diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index d7d82800b8b55..3dfd1f38df76f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -20,6 +20,8 @@ package org.apache.spark.deploy.history import java.util.zip.ZipOutputStream import scala.xml.Node +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.health.HealthCheckRegistry import org.apache.spark.SparkException import org.apache.spark.ui.SparkUI @@ -72,6 +74,11 @@ private[history] case class LoadedAppUI( ui: SparkUI, updateProbe: () => Boolean) + +private[history] case class ApplicationHistoryBinding(metrics: MetricRegistry, + health: HealthCheckRegistry) { +} + private[history] abstract class ApplicationHistoryProvider { /** @@ -98,6 +105,13 @@ private[history] abstract class ApplicationHistoryProvider { return 0; } + /** + * Bind to the History Server: threads should be started here; exceptions may be raised + * if the history provider cannot be started. + * @param binding binding information + */ + def start(binding: ApplicationHistoryBinding): Unit = {} + /** * Returns a list of applications available for the history server to show. * diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 9012736bc2745..13c0cffc285b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -22,6 +22,9 @@ import java.util.UUID import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} +import com.codahale.metrics.{Timer, Counter, MetricRegistry} +import org.apache.spark.metrics.source.Source + import scala.collection.mutable import scala.xml.Node @@ -128,6 +131,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val pendingReplayTasksCount = new java.util.concurrent.atomic.AtomicInteger(0) + private val historyMetrics = new HistoryMetrics() + /** * Return a runnable that performs the given operation on the event logs. * This operation is expected to be executed periodically. @@ -212,7 +217,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } throw new FileNotFoundException(msg).initCause(f) } + } + + /** + * Bind to the History Server: start update and cleaner threads; perform any metric registration + * @param binding binding information + */ + override def start(binding: ApplicationHistoryBinding): Unit = { + super.start(binding) + binding.metrics.register(MetricRegistry.name(historyMetrics.sourceName), + historyMetrics.metricRegistry) // Disable the background thread during tests. if (!conf.contains("spark.testing")) { // A task that periodically checks for event log updates on disk. @@ -239,6 +254,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { + val time = historyMetrics.appUiLoadTimer.time() try { applications.get(appId).flatMap { appInfo => appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => @@ -274,6 +290,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } catch { case e: FileNotFoundException => None + } finally { + time.stop() } } @@ -310,6 +328,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * applications that haven't been updated since last time the logs were checked. */ private[history] def checkForLogs(): Unit = { + historyMetrics.updateCount.inc() + val updateContext = historyMetrics.updateTimer.time() try { val newLastScanTime = getNewLastScanTime() logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") @@ -381,6 +401,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) lastScanTime.set(newLastScanTime) } catch { case e: Exception => logError("Exception in checking for event log updates", e) + historyMetrics.updateFailureCount.inc() + + } finally { + updateContext.stop() } } @@ -680,6 +704,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** +<<<<<<< 64515e5fbfd694d06fdbc28040fce7baf90a32aa * String description for diagnostics * @return a summary of the component state */ @@ -729,6 +754,41 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) prevFileSize < latest.fileSize } } + + /** + * Metrics integration: the various counters of activity + */ + private[history] class HistoryMetrics extends Source { + + /** Name for metrics: yarn_history */ + override val sourceName = "history.fs" + + /** Metrics registry */ + override val metricRegistry = new MetricRegistry() + + /** Number of updates */ + val updateCount = new Counter() + + /** Number of updates */ + val updateFailureCount = new Counter() + + /** Statistics on update duration */ + val updateTimer = new Timer() + + /** Statistics on time to load app UIs */ + val appUiLoadTimer = new Timer() + + /** + * Register metrics. + */ + def registerMetrics(): Unit = { + metricRegistry.register("update.count", updateCount) + metricRegistry.register("update.failure.count", updateFailureCount) + metricRegistry.register("update.time", updateTimer) + metricRegistry.register("appui.load.time", appUiLoadTimer) + } + } + } private[history] object FsHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 54f39f7620e5d..0855996a6793c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -24,6 +24,13 @@ import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.util.control.NonFatal import scala.xml.Node +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.health.HealthCheckRegistry +import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import com.codahale.metrics.servlets.HealthCheckServlet +import com.codahale.metrics.servlets.MetricsServlet +import com.codahale.metrics.servlets.ThreadDumpServlet +import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} @@ -62,6 +69,20 @@ class HistoryServer( // application private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) + private[history] val metrics = new MetricRegistry() + private[history] val health = new HealthCheckRegistry() + + private val appLoader = new CacheLoader[String, SparkUI] { + override def load(key: String): SparkUI = { + val parts = key.split("/") + require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") + val ui = provider + .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) + .getOrElse(throw new NoSuchElementException(s"no app with key $key")) + attachSparkUI(ui) + ui + } + } // and its metrics, for testing as well as monitoring val cacheMetrics = appCache.metrics @@ -129,6 +150,20 @@ class HistoryServer( contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") attachHandler(contextHandler) + + // hook up Codahale metrics + val registry = metrics + val metricsHandler = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) + metricsHandler.addServlet(new ServletHolder(new HealthCheckServlet(health)), + HistoryServer.METRICS_PATH_PREFIX) + metricsHandler.addServlet(new ServletHolder(new ThreadDumpServlet()), + HistoryServer.THREADS_SUBPATH) + metricsHandler.addServlet(new ServletHolder(new MetricsServlet(registry)), + HistoryServer.HEALTH_SUBPATH) + attachHandler(metricsHandler) + registry.registerAll(new ThreadStatesGaugeSet()); + registry.registerAll(new MemoryUsageGaugeSet()); + registry.registerAll(new GarbageCollectorMetricSet()); } /** Bind to the HTTP server behind this web interface. */ @@ -264,6 +299,9 @@ object HistoryServer extends Logging { private val conf = new SparkConf val UI_PATH_PREFIX = "/history" + val METRICS_PATH_PREFIX = "/metrics" + val HEALTH_SUBPATH = "/health" + val THREADS_SUBPATH = "/threads" def main(argStrings: Array[String]): Unit = { Utils.initDaemon(log) @@ -282,6 +320,8 @@ object HistoryServer extends Logging { val server = new HistoryServer(conf, provider, securityManager, port) server.bind() + val binding = new ApplicationHistoryBinding(server.metrics, server.health) + provider.start(binding) ShutdownHookManager.addShutdownHook { () => server.stop() } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 1d494500cdb5c..ab0d83c07902f 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import com.codahale.metrics.{MetricRegistry, Metric, MetricFilter} import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} @@ -211,6 +211,8 @@ private[spark] class MetricsSystem private ( } } } + + private[spark] def getMetricRegistry(): MetricRegistry = { registry } } private[spark] object MetricsSystem { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index ec580a44b8e76..9ccc83e3c3d41 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -23,6 +23,9 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.health.HealthCheckRegistry + import scala.concurrent.duration._ import scala.language.postfixOps @@ -45,9 +48,13 @@ import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { private var testDir: File = null + private var metrics: MetricRegistry = _ + private var health: HealthCheckRegistry = _ before { testDir = Utils.createTempDir(namePrefix = s"a b%20c+d") + metrics = new MetricRegistry() + health = new HealthCheckRegistry() } after { @@ -69,6 +76,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("Parse application logs") { val clock = new ManualClock(12345678) val provider = new FsHistoryProvider(createTestConf(), clock) + val provider = createHistoryProvider() // Write a new-style application log. val newAppComplete = newLogFile("new1", None, inProgress = false) @@ -130,6 +138,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + private def createHistoryProvider(): FsHistoryProvider = { + val provider = new FsHistoryProvider(createTestConf()) + val binding = new ApplicationHistoryBinding(metrics, health) + provider.start(binding) + provider + } + test("SPARK-3697: ignore directories that cannot be read.") { // setReadable(...) does not work on Windows. Please refer JDK-6728842. assume(!Utils.isWindows) @@ -145,14 +160,14 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc ) logFile2.setReadable(false, false) - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() updateAndCheck(provider) { list => list.size should be (1) } } test("history file is renamed from inprogress to completed") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, @@ -186,7 +201,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("SPARK-5582: empty log directory") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, @@ -202,7 +217,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("apps with multiple attempts with order") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = true) writeFile(attempt1, true, None, @@ -347,7 +362,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("Event log copy") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() val logs = (1 to 2).map { i => val log = newLogFile("downloadApp1", Some(s"attempt$i"), inProgress = false) writeFile(log, true, None, @@ -382,7 +397,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("SPARK-8372: new logs with no app ID are ignored") { - val provider = new FsHistoryProvider(createTestConf()) + val provider = createHistoryProvider() // Write a new log file without an app id, to make sure it's ignored. val logFile1 = newLogFile("app1", None, inProgress = true) diff --git a/pom.xml b/pom.xml index c1174593c1922..c66f6358655ef 100644 --- a/pom.xml +++ b/pom.xml @@ -592,6 +592,16 @@ metrics-graphite ${codahale.metrics.version} + + io.dropwizard.metrics + metrics-healthchecks + ${codahale.metrics.version} + + + io.dropwizard.metrics + metrics-servlets + ${codahale.metrics.version} + com.fasterxml.jackson.core jackson-databind From 0825756fdc29eb36842ea575d4fbefc6093c68ab Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 9 Nov 2015 18:18:51 +0000 Subject: [PATCH 02/22] [SPARK-11373] tests and review; found and fixed a re-entrancy in HistoryServerSuite which was causing problems with counter registration --- .../history/ApplicationHistoryProvider.scala | 9 ++- .../deploy/history/FsHistoryProvider.scala | 79 ++++++++++++++----- .../spark/deploy/history/HistoryServer.scala | 69 +++++++++------- .../history/FsHistoryProviderSuite.scala | 11 +++ .../deploy/history/HistoryServerSuite.scala | 2 +- 5 files changed, 116 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 3dfd1f38df76f..f6136da124869 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -81,6 +81,8 @@ private[history] case class ApplicationHistoryBinding(metrics: MetricRegistry, private[history] abstract class ApplicationHistoryProvider { + private var binding: Option[ApplicationHistoryBinding] = None + /** * Returns the count of application event logs that the provider is currently still processing. * History Server UI can use this to indicate to a user that the application listing on the UI @@ -108,9 +110,12 @@ private[history] abstract class ApplicationHistoryProvider { /** * Bind to the History Server: threads should be started here; exceptions may be raised * if the history provider cannot be started. - * @param binding binding information + * @param historyBinding binding information */ - def start(binding: ApplicationHistoryBinding): Unit = {} + def start(historyBinding: ApplicationHistoryBinding): Unit = { + require(binding.isEmpty, "History provider already started") + binding = Some(historyBinding) + } /** * Returns a list of applications available for the history server to show. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 13c0cffc285b1..76226dcc6402a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -22,7 +22,8 @@ import java.util.UUID import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} -import com.codahale.metrics.{Timer, Counter, MetricRegistry} +import com.codahale.metrics.health.HealthCheckRegistry +import com.codahale.metrics._ import org.apache.spark.metrics.source.Source import scala.collection.mutable @@ -217,17 +218,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } throw new FileNotFoundException(msg).initCause(f) } - } - - /** - * Bind to the History Server: start update and cleaner threads; perform any metric registration - * @param binding binding information - */ - override def start(binding: ApplicationHistoryBinding): Unit = { - super.start(binding) - binding.metrics.register(MetricRegistry.name(historyMetrics.sourceName), - historyMetrics.metricRegistry) // Disable the background thread during tests. if (!conf.contains("spark.testing")) { // A task that periodically checks for event log updates on disk. @@ -243,6 +234,16 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + /** + * Bind to the History Server: start update and cleaner threads; perform any metric registration + * + * @param binding binding information + */ + override def start(binding: ApplicationHistoryBinding): Unit = { + super.start(binding) + historyMetrics.registerMetrics(binding.metrics, binding.health) + } + override def getListing(): Iterator[FsApplicationHistoryInfo] = applications.values.iterator override def getApplicationInfo(appId: String): Option[FsApplicationHistoryInfo] = { @@ -254,7 +255,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { - val time = historyMetrics.appUiLoadTimer.time() + historyMetrics.appUILoadCount.inc() + val timer = historyMetrics.appUiLoadTimer.time() try { applications.get(appId).flatMap { appInfo => appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => @@ -289,9 +291,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } } catch { - case e: FileNotFoundException => None + case e: FileNotFoundException => + historyMetrics.appUILoadFailureCount.inc() + None + case other: Exception => + historyMetrics.appUILoadFailureCount.inc() + throw other } finally { - time.stop() + timer.stop() } } @@ -320,6 +327,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) initThread.interrupt() initThread.join() } + historyMetrics.unregister() } /** @@ -760,32 +768,61 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private[history] class HistoryMetrics extends Source { - /** Name for metrics: yarn_history */ + /** Name for metrics: `history.fs` */ override val sourceName = "history.fs" + private val name = MetricRegistry.name(sourceName) + /** Metrics registry */ override val metricRegistry = new MetricRegistry() /** Number of updates */ val updateCount = new Counter() - /** Number of updates */ + /** Number of update failures */ val updateFailureCount = new Counter() + /** Number of App UI load operations */ + val appUILoadCount = new Counter() + + /** Number of App UI load operations that failed*/ + val appUILoadFailureCount = new Counter() + /** Statistics on update duration */ val updateTimer = new Timer() /** Statistics on time to load app UIs */ val appUiLoadTimer = new Timer() + private var metrics: Option[MetricRegistry] = None + private var health: Option[HealthCheckRegistry] = None + /** * Register metrics. */ - def registerMetrics(): Unit = { - metricRegistry.register("update.count", updateCount) - metricRegistry.register("update.failure.count", updateFailureCount) - metricRegistry.register("update.time", updateTimer) - metricRegistry.register("appui.load.time", appUiLoadTimer) + def registerMetrics(metrics: MetricRegistry, health: HealthCheckRegistry): Unit = { + + this.metrics = Some(metrics) + this.health = Some(health) + def register(s: String, m: Metric) = { + metricRegistry.register(s, m) + } + register("update.count", updateCount) + register("update.failure.count", updateFailureCount) + register("update.time", updateTimer) + register("appui.load.time", appUiLoadTimer) + register("appui.load.count", appUILoadCount) + register("appui.load.failure.count", appUILoadFailureCount) + metrics.register(name, metricRegistry) + } + + /** + * Unregister + */ + def unregister(): Unit = { + metrics.foreach(_.removeMatching(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name.startsWith(name) + })) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 0855996a6793c..62beb8b584e09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.history import java.util.NoSuchElementException +import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} @@ -69,6 +70,7 @@ class HistoryServer( // application private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) + private val initialized = new AtomicBoolean(false) private[history] val metrics = new MetricRegistry() private[history] val health = new HealthCheckRegistry() @@ -136,34 +138,36 @@ class HistoryServer( /** * Initialize the history server. * - * This starts a background thread that periodically synchronizes information displayed on - * this UI with the event logs in the provided base directory. + * This calls [[ApplicationHistoryProvider.start()]] to start the history provider. */ def initialize() { - attachPage(new HistoryPage(this)) - - attachHandler(ApiRootResource.getServletHandler(this)) - - attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - - val contextHandler = new ServletContextHandler - contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) - contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") - attachHandler(contextHandler) - - // hook up Codahale metrics - val registry = metrics - val metricsHandler = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) - metricsHandler.addServlet(new ServletHolder(new HealthCheckServlet(health)), - HistoryServer.METRICS_PATH_PREFIX) - metricsHandler.addServlet(new ServletHolder(new ThreadDumpServlet()), - HistoryServer.THREADS_SUBPATH) - metricsHandler.addServlet(new ServletHolder(new MetricsServlet(registry)), - HistoryServer.HEALTH_SUBPATH) - attachHandler(metricsHandler) - registry.registerAll(new ThreadStatesGaugeSet()); - registry.registerAll(new MemoryUsageGaugeSet()); - registry.registerAll(new GarbageCollectorMetricSet()); + if (!initialized.getAndSet(true)) { + attachPage(new HistoryPage(this)) + + attachHandler(ApiRootResource.getServletHandler(this)) + + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + + val contextHandler = new ServletContextHandler + contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) + contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") + attachHandler(contextHandler) + + // hook up Codahale metrics + val metricsHandler = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) + metricsHandler.addServlet(new ServletHolder(new HealthCheckServlet(health)), + HistoryServer.METRICS_PATH_PREFIX) + metricsHandler.addServlet(new ServletHolder(new ThreadDumpServlet()), + HistoryServer.THREADS_SUBPATH) + metricsHandler.addServlet(new ServletHolder(new MetricsServlet(metrics)), + HistoryServer.HEALTH_SUBPATH) + attachHandler(metricsHandler) + metrics.registerAll(new ThreadStatesGaugeSet()); + metrics.registerAll(new MemoryUsageGaugeSet()); + metrics.registerAll(new GarbageCollectorMetricSet()); + // start the provider against the metrics binding + provider.start(new ApplicationHistoryBinding(metrics, health)) + } } /** Bind to the HTTP server behind this web interface. */ @@ -171,11 +175,18 @@ class HistoryServer( super.bind() } - /** Stop the server and close the file system. */ + /** Stop the server and close the history provider. */ override def stop() { super.stop() provider.stop() - appCache.stop() + try { + super.stop() + } finally { + appCache.stop() + if (provider != null) { + provider.stop() + } + } } /** Attach a reconstructed UI to this server. Only valid after bind(). */ @@ -320,8 +331,6 @@ object HistoryServer extends Logging { val server = new HistoryServer(conf, provider, securityManager, port) server.bind() - val binding = new ApplicationHistoryBinding(server.metrics, server.health) - provider.start(binding) ShutdownHookManager.addShutdownHook { () => server.stop() } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 9ccc83e3c3d41..bb48a0ae5b4aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -73,6 +73,17 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc new File(logPath) } + /** + * Create a configure a new history provider + * @return a filesystem history provider ready for use + */ + private def createHistoryProvider(): FsHistoryProvider = { + val provider = new FsHistoryProvider(createTestConf()) + val binding = new ApplicationHistoryBinding(metrics, health) + provider.start(binding) + provider + } + test("Parse application logs") { val clock = new ManualClock(12345678) val provider = new FsHistoryProvider(createTestConf(), clock) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 764156c3edc41..3cc98d397f5c2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -80,9 +80,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val securityManager = HistoryServer.createSecurityManager(conf) server = new HistoryServer(conf, provider, securityManager, 18080) - server.initialize() server.bind() port = server.boundPort + provider.checkForLogs() } def stop(): Unit = { From 9e63bcf0047466731f8c3854fdb8c7abeb8c160f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 9 Nov 2015 19:26:30 +0000 Subject: [PATCH 03/22] [SPARK-11373] scala-style javadoc tuning in FsHistoryProvider --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 76226dcc6402a..e5e9eb6fbcba3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -785,7 +785,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** Number of App UI load operations */ val appUILoadCount = new Counter() - /** Number of App UI load operations that failed*/ + /** Number of App UI load operations that failed */ val appUILoadFailureCount = new Counter() /** Statistics on update duration */ From a658769401e8d7f3c07f38eb5f1b30113348413d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 26 Nov 2015 21:18:38 +0000 Subject: [PATCH 04/22] SPARK-11373: move to MetricsSystem, though retaining/expanding health checks, which aren't part of it --- .../history/ApplicationHistoryProvider.scala | 13 +- .../deploy/history/FsHistoryProvider.scala | 364 ++++++++++-------- .../spark/deploy/history/HistoryMetrics.scala | 49 +++ .../spark/deploy/history/HistoryServer.scala | 45 ++- .../history/FsHistoryProviderSuite.scala | 2 +- 5 files changed, 297 insertions(+), 176 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index f6136da124869..46120826565ea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -24,6 +24,7 @@ import com.codahale.metrics.MetricRegistry import com.codahale.metrics.health.HealthCheckRegistry import org.apache.spark.SparkException +import org.apache.spark.metrics.source.Source import org.apache.spark.ui.SparkUI private[spark] case class ApplicationAttemptInfo( @@ -74,9 +75,11 @@ private[history] case class LoadedAppUI( ui: SparkUI, updateProbe: () => Boolean) - -private[history] case class ApplicationHistoryBinding(metrics: MetricRegistry, - health: HealthCheckRegistry) { +/** + * Binding information. Initially empty, may expand over time: it's here so that + * subclassed providers do not break if binding information is added/expanded + */ +private[history] case class ApplicationHistoryBinding() { } private[history] abstract class ApplicationHistoryProvider { @@ -111,10 +114,12 @@ private[history] abstract class ApplicationHistoryProvider { * Bind to the History Server: threads should be started here; exceptions may be raised * if the history provider cannot be started. * @param historyBinding binding information + * @return the metric and binding information for registration */ - def start(historyBinding: ApplicationHistoryBinding): Unit = { + def start(historyBinding: ApplicationHistoryBinding): (Option[Source], Option[HealthCheckSource]) = { require(binding.isEmpty, "History provider already started") binding = Some(historyBinding) + (None, None) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e5e9eb6fbcba3..23a216b0c505d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -22,7 +22,8 @@ import java.util.UUID import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} -import com.codahale.metrics.health.HealthCheckRegistry +import com.codahale.metrics.health.HealthCheck.Result +import com.codahale.metrics.health.{HealthCheck, HealthCheckRegistry} import com.codahale.metrics._ import org.apache.spark.metrics.source.Source @@ -132,7 +133,11 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val pendingReplayTasksCount = new java.util.concurrent.atomic.AtomicInteger(0) - private val historyMetrics = new HistoryMetrics() + /** filesystem metrics: visible for test access */ + private[history] val metrics = new FsHistoryProviderMetrics(this) + + /** filesystem health: visible for test access */ + private[history] val health = new FsHistoryProviderHealth(this) /** * Return a runnable that performs the given operation on the event logs. @@ -169,6 +174,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + /** + * Bind to the History Server: threads should be started here; exceptions may be raised + * if the history provider cannot be started. + * @param historyBinding binding information + * @return the metric and binding information for registration + */ + override def start(historyBinding: ApplicationHistoryBinding) = { + super.start(historyBinding) + (Some(metrics), Some(health)) + } + private[history] def startSafeModeCheckThread( errorHandler: Option[Thread.UncaughtExceptionHandler]): Thread = { // Cannot probe anything while the FS is in safe mode, so spawn a new thread that will wait @@ -255,50 +271,49 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { - historyMetrics.appUILoadCount.inc() - val timer = historyMetrics.appUiLoadTimer.time() - try { - applications.get(appId).flatMap { appInfo => - appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => - val replayBus = new ReplayListenerBus() - val ui = { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - SparkUI.createHistoryUI(conf, replayBus, appSecManager, appInfo.name, - HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) - // Do not call ui.bind() to avoid creating a new server for each application - } + metrics.appUILoadCount.inc() + time(metrics.appUiLoadTimer) { + try { + applications.get(appId).flatMap { appInfo => + appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => + val replayBus = new ReplayListenerBus() + val ui = { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + SparkUI.createHistoryUI(conf, replayBus, appSecManager, appInfo.name, + HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) + // Do not call ui.bind() to avoid creating a new server for each application + } + + val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) + + val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) + + if (appListener.appId.isDefined) { + ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) + // make sure to set admin acls before view acls so they are properly picked up + val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") + ui.getSecurityManager.setAdminAcls(adminAcls) + ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) + val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + + appListener.adminAclsGroups.getOrElse("") + ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) + ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) + Some(LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize))) + } else { + None + } - val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) - - val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) - - if (appListener.appId.isDefined) { - ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) - // make sure to set admin acls before view acls so they are properly picked up - val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") - ui.getSecurityManager.setAdminAcls(adminAcls) - ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) - val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - appListener.adminAclsGroups.getOrElse("") - ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) - ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) - Some(LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize))) - } else { - None } - } + } catch { + case e: FileNotFoundException => + metrics.appUILoadNotFoundCount.inc() + None + case other: Exception => + metrics.appUILoadFailureCount.inc() + throw other } - } catch { - case e: FileNotFoundException => - historyMetrics.appUILoadFailureCount.inc() - None - case other: Exception => - historyMetrics.appUILoadFailureCount.inc() - throw other - } finally { - timer.stop() } } @@ -327,7 +342,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) initThread.interrupt() initThread.join() } - historyMetrics.unregister() } /** @@ -336,83 +350,87 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * applications that haven't been updated since last time the logs were checked. */ private[history] def checkForLogs(): Unit = { - historyMetrics.updateCount.inc() - val updateContext = historyMetrics.updateTimer.time() - try { - val newLastScanTime = getNewLastScanTime() - logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") - val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) - .getOrElse(Seq[FileStatus]()) - // scan for modified applications, replay and merge them - val logInfos: Seq[FileStatus] = statusList - .filter { entry => - try { - val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L) - !entry.isDirectory() && - // FsHistoryProvider generates a hidden file which can't be read. Accidentally - // reading a garbage file is safe, but we would log an error which can be scary to - // the end-user. - !entry.getPath().getName().startsWith(".") && - prevFileSize < entry.getLen() - } catch { - case e: AccessControlException => - // Do not use "logInfo" since these messages can get pretty noisy if printed on - // every poll. - logDebug(s"No permission to read $entry, ignoring.") - false + metrics.updateCount.inc() + time(metrics.updateTimer) { + try { + val newLastScanTime = getNewLastScanTime() + logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") + val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) + .getOrElse(Seq[FileStatus]()) + // scan for modified applications, replay and merge them + val logInfos: Seq[FileStatus] = statusList + .filter { entry => + try { + val prevFileSize = fileToAppInfo.get(entry.getPath()) .map {_.fileSize}.getOrElse(0L) + !entry.isDirectory() && + // FsHistoryProvider generates a hidden file which can't be read. Accidentally + // reading a garbage file is safe, but we would log an error which can be scary to + // the end-user. + !entry.getPath().getName().startsWith(".") && + prevFileSize < entry.getLen() + } catch { + case e: AccessControlException => + // Do not use "logInfo" since these messages can get pretty noisy if printed on + // every poll. + logDebug(s"No permission to read $entry, ignoring.") + false + } + } + .flatMap { entry => Some(entry) } + .sortWith { case (entry1, entry2) => + entry1.getModificationTime() >= entry2.getModificationTime() } - } - .flatMap { entry => Some(entry) } - .sortWith { case (entry1, entry2) => - entry1.getModificationTime() >= entry2.getModificationTime() - } if (logInfos.nonEmpty) { logDebug(s"New/updated attempts found: ${logInfos.size} ${logInfos.map(_.getPath)}") } - var tasks = mutable.ListBuffer[Future[_]]() - - try { - for (file <- logInfos) { - tasks += replayExecutor.submit(new Runnable { - override def run(): Unit = mergeApplicationListing(file) - }) - } - } catch { - // let the iteration over logInfos break, since an exception on - // replayExecutor.submit (..) indicates the ExecutorService is unable - // to take any more submissions at this time - - case e: Exception => - logError(s"Exception while submitting event log for replay", e) - } - - pendingReplayTasksCount.addAndGet(tasks.size) + var tasks = mutable.ListBuffer[Future[_]]() - tasks.foreach { task => try { - // Wait for all tasks to finish. This makes sure that checkForLogs - // is not scheduled again while some tasks are already running in - // the replayExecutor. - task.get() + for (file <- logInfos) { + tasks += replayExecutor.submit(new Runnable { + override def run(): Unit = time(metrics.mergeApplicationListingTimer) { + mergeApplicationListing(file) + } + }) + } } catch { - case e: InterruptedException => - throw e + // let the iteration over logInfos break, since an exception on + // replayExecutor.submit (..) indicates the ExecutorService is unable + // to take any more submissions at this time + case e: Exception => - logError("Exception while merging application listings", e) - } finally { - pendingReplayTasksCount.decrementAndGet() + logError(s"Exception while submitting event log for replay", e) } - } - lastScanTime.set(newLastScanTime) - } catch { - case e: Exception => logError("Exception in checking for event log updates", e) - historyMetrics.updateFailureCount.inc() + pendingReplayTasksCount.addAndGet(tasks.size) - } finally { - updateContext.stop() + tasks.foreach { task => + try { + // Wait for all tasks to finish. This makes sure that checkForLogs + // is not scheduled again while some tasks are already running in + // the replayExecutor. + task.get() + } catch { + case e: InterruptedException => + throw e + case e: Exception => + logError("Exception while merging application listings", e) + } finally { + pendingReplayTasksCount.decrementAndGet() + } + } + + lastScanTime.set(newLastScanTime) + } catch { + case e: Exception => logError( + "Exception in checking for event log updates", + e) + metrics.updateFailureCount.inc() + } finally { + updateContext.stop() + } } } @@ -712,7 +730,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** -<<<<<<< 64515e5fbfd694d06fdbc28040fce7baf90a32aa * String description for diagnostics * @return a summary of the component state */ @@ -765,67 +782,106 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** * Metrics integration: the various counters of activity + * Time a closure, returning its output. + * @param t timer + * @param f function + * @tparam T type of return value of time + * @return the result of the function. */ - private[history] class HistoryMetrics extends Source { + private def time[T](t: Timer)(f: => T): T = { + val timeCtx = t.time() + try { + f + } finally { + timeCtx.close() + } + } +} - /** Name for metrics: `history.fs` */ - override val sourceName = "history.fs" +/** + * Health checks + */ +private[history] class FsHistoryProviderHealth(owner: FsHistoryProvider) extends HealthCheckSource { - private val name = MetricRegistry.name(sourceName) + override val sourceName = "history.fs" - /** Metrics registry */ - override val metricRegistry = new MetricRegistry() + private val name = MetricRegistry.name(sourceName) + val healthRegistry = new HealthCheckRegistry - /** Number of updates */ - val updateCount = new Counter() + val fileSystemLive = new HealthCheck { + override def check() = { + if (owner.isFsInSafeMode()) { + Result.unhealthy("Filesystem is in safe mode") + } else { + Result.healthy() + } + } + } - /** Number of update failures */ - val updateFailureCount = new Counter() + private val healthChecks = Seq( + ("filesystem.healthy", fileSystemLive) + ) - /** Number of App UI load operations */ - val appUILoadCount = new Counter() + healthChecks.foreach(elt => healthRegistry.register(elt._1, elt._2)) - /** Number of App UI load operations that failed */ - val appUILoadFailureCount = new Counter() +} + /** + * Metrics integration: the various counters of activity + */ +private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extends Source { - /** Statistics on update duration */ - val updateTimer = new Timer() + override val sourceName = "history.fs" - /** Statistics on time to load app UIs */ - val appUiLoadTimer = new Timer() + private val name = MetricRegistry.name(sourceName) - private var metrics: Option[MetricRegistry] = None - private var health: Option[HealthCheckRegistry] = None + /** Metrics registry */ + override val metricRegistry = new MetricRegistry() - /** - * Register metrics. - */ - def registerMetrics(metrics: MetricRegistry, health: HealthCheckRegistry): Unit = { + /** Number of updates */ + val updateCount = new Counter() - this.metrics = Some(metrics) - this.health = Some(health) - def register(s: String, m: Metric) = { - metricRegistry.register(s, m) - } - register("update.count", updateCount) - register("update.failure.count", updateFailureCount) - register("update.time", updateTimer) - register("appui.load.time", appUiLoadTimer) - register("appui.load.count", appUILoadCount) - register("appui.load.failure.count", appUILoadFailureCount) - metrics.register(name, metricRegistry) - } + /** Number of update failures */ + val updateFailureCount = new Counter() - /** - * Unregister - */ - def unregister(): Unit = { - metrics.foreach(_.removeMatching(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name.startsWith(name) - })) - } - } + /** Number of App UI load operations */ + val appUILoadCount = new Counter() + + /** Number of App UI load operations that failed */ + val appUILoadFailureCount = new Counter() + + /** Number of App UI load operations that failed due to an unknown file */ + val appUILoadNotFoundCount = new Counter() + /** Statistics on update duration */ + val updateTimer = new Timer() + + /** Statistics on time to load app UIs */ + val appUiLoadTimer = new Timer() + + /** Statistics on time to replay and merge listings */ + val mergeApplicationListingTimer = new Timer() + + private val counters = Seq( + ("update.count", updateCount), + ("update.failure.count", updateFailureCount), + ("appui.load.count", appUILoadCount), + ("appui.load.failure.count", appUILoadFailureCount)) + ("appui.load.not-found.count", appUILoadNotFoundCount) + + private val timers = Seq ( + ("update.timer", updateTimer), + ("merge.application.listings.timer", mergeApplicationListingTimer), + ("appui.load.timer", appUiLoadTimer)) + + val allMetrics = counters ++ timers + + allMetrics.foreach(elt => metricRegistry.register(elt._1, elt._2)) + + override def toString: String = { + def sb = new StringBuilder(counters.size * 20) + allMetrics.foreach(elt => sb.append(s" ${elt._1} = ${elt._2}\n")) + sb.toString() + } } private[history] object FsHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala new file mode 100644 index 0000000000000..e780b2bfea45a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.health.HealthCheckRegistry + +import org.apache.spark.metrics.source.Source + +private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { + override val metricRegistry = new MetricRegistry() + override val sourceName = "history" + +/* // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers"), new Gauge[Int] { + override def getValue: Int = master.workers.size + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps"), new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waitingApps"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + })*/ +} + +private[spark] trait HealthCheckSource { + def sourceName: String + + def healthRegistry: HealthCheckRegistry +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 62beb8b584e09..060edaf744cc2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -28,13 +28,15 @@ import scala.xml.Node import com.codahale.metrics.MetricRegistry import com.codahale.metrics.health.HealthCheckRegistry import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} +//import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} import com.codahale.metrics.servlets.HealthCheckServlet -import com.codahale.metrics.servlets.MetricsServlet +//import com.codahale.metrics.servlets.MetricsServlet import com.codahale.metrics.servlets.ThreadDumpServlet import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -71,8 +73,11 @@ class HistoryServer( // application private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) private val initialized = new AtomicBoolean(false) - private[history] val metrics = new MetricRegistry() - private[history] val health = new HealthCheckRegistry() + + private[history] val metricsSystem: MetricsSystem = MetricsSystem.createMetricsSystem("history", + conf, securityManager) + private[history] var metricsReg = metricsSystem.getMetricRegistry + private[history] var healthChecks: Option[HealthCheckSource] = _ private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { @@ -153,20 +158,23 @@ class HistoryServer( contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") attachHandler(contextHandler) - // hook up Codahale metrics - val metricsHandler = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) - metricsHandler.addServlet(new ServletHolder(new HealthCheckServlet(health)), - HistoryServer.METRICS_PATH_PREFIX) - metricsHandler.addServlet(new ServletHolder(new ThreadDumpServlet()), - HistoryServer.THREADS_SUBPATH) - metricsHandler.addServlet(new ServletHolder(new MetricsServlet(metrics)), - HistoryServer.HEALTH_SUBPATH) - attachHandler(metricsHandler) - metrics.registerAll(new ThreadStatesGaugeSet()); - metrics.registerAll(new MemoryUsageGaugeSet()); - metrics.registerAll(new GarbageCollectorMetricSet()); + // hook up metrics + // start the provider against the metrics binding - provider.start(new ApplicationHistoryBinding(metrics, health)) + val (source, health) = provider.start(new ApplicationHistoryBinding()) + val codahaleContext = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) + codahaleContext.addServlet(new ServletHolder(new ThreadDumpServlet()), + HistoryServer.THREADS_SUBPATH) + health.foreach { healthSource => + codahaleContext.addServlet( + new ServletHolder(new HealthCheckServlet(healthSource.healthRegistry)), + HistoryServer.HEALTH_SUBPATH) + } + healthChecks = health + attachHandler(codahaleContext) + source.foreach(metricsSystem.registerSource) + metricsSystem.start() + metricsSystem.getServletHandlers.foreach(attachHandler) } } @@ -186,6 +194,9 @@ class HistoryServer( if (provider != null) { provider.stop() } + if (metricsSystem != null) { + metricsSystem.stop() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index bb48a0ae5b4aa..1b5899a37272a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -79,7 +79,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc */ private def createHistoryProvider(): FsHistoryProvider = { val provider = new FsHistoryProvider(createTestConf()) - val binding = new ApplicationHistoryBinding(metrics, health) + val binding = new ApplicationHistoryBinding() provider.start(binding) provider } From f02323fc3ecab8ec77f3f67f3959cf4ad83142be Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 26 Nov 2015 21:33:28 +0000 Subject: [PATCH 05/22] scalacheck --- .../history/ApplicationHistoryProvider.scala | 3 ++- .../deploy/history/FsHistoryProvider.scala | 3 ++- .../spark/deploy/history/HistoryMetrics.scala | 22 ++++++------------- .../spark/deploy/history/HistoryServer.scala | 2 +- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 46120826565ea..ab1c7aa93b794 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -116,7 +116,8 @@ private[history] abstract class ApplicationHistoryProvider { * @param historyBinding binding information * @return the metric and binding information for registration */ - def start(historyBinding: ApplicationHistoryBinding): (Option[Source], Option[HealthCheckSource]) = { + def start(historyBinding: ApplicationHistoryBinding): + (Option[Source], Option[HealthCheckSource]) = { require(binding.isEmpty, "History provider already started") binding = Some(historyBinding) (None, None) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 23a216b0c505d..0c3508ec80b74 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -180,7 +180,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * @param historyBinding binding information * @return the metric and binding information for registration */ - override def start(historyBinding: ApplicationHistoryBinding) = { + override def start(historyBinding: ApplicationHistoryBinding): + (Option[Source], Option[HealthCheckSource]) = { super.start(historyBinding) (Some(metrics), Some(health)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala index e780b2bfea45a..50ef12db474c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala @@ -22,26 +22,18 @@ import com.codahale.metrics.health.HealthCheckRegistry import org.apache.spark.metrics.source.Source +/** + * History system metrics independent of providers go in here + * @param owner owning instance + */ private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "history" - -/* // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name("workers"), new Gauge[Int] { - override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps"), new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waitingApps"), new Gauge[Int] { - override def getValue: Int = master.waitingApps.size - })*/ } +/** + * A trait for sources of health checks to implement + */ private[spark] trait HealthCheckSource { def sourceName: String diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 060edaf744cc2..7b7c1c422fef1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -30,7 +30,6 @@ import com.codahale.metrics.health.HealthCheckRegistry import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} //import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} import com.codahale.metrics.servlets.HealthCheckServlet -//import com.codahale.metrics.servlets.MetricsServlet import com.codahale.metrics.servlets.ThreadDumpServlet import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -173,6 +172,7 @@ class HistoryServer( healthChecks = health attachHandler(codahaleContext) source.foreach(metricsSystem.registerSource) + metricsSystem.registerSource(new HistoryMetrics(this)) metricsSystem.start() metricsSystem.getServletHandlers.foreach(attachHandler) } From dbde3ab8ccb8516584bcde2a46df588c62133bd8 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 26 Nov 2015 21:57:42 +0000 Subject: [PATCH 06/22] SPARK-11373: cut out the notion of binding information; simplify provider launch --- .../history/ApplicationHistoryProvider.scala | 20 +++++++------------ .../deploy/history/FsHistoryProvider.scala | 8 +++----- .../spark/deploy/history/HistoryServer.scala | 2 +- .../history/FsHistoryProviderSuite.scala | 3 +-- 4 files changed, 12 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index ab1c7aa93b794..b43addd44420e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.history +import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.ZipOutputStream import scala.xml.Node @@ -75,16 +76,9 @@ private[history] case class LoadedAppUI( ui: SparkUI, updateProbe: () => Boolean) -/** - * Binding information. Initially empty, may expand over time: it's here so that - * subclassed providers do not break if binding information is added/expanded - */ -private[history] case class ApplicationHistoryBinding() { -} - private[history] abstract class ApplicationHistoryProvider { - private var binding: Option[ApplicationHistoryBinding] = None + private val started = new AtomicBoolean(false) /** * Returns the count of application event logs that the provider is currently still processing. @@ -112,14 +106,14 @@ private[history] abstract class ApplicationHistoryProvider { /** * Bind to the History Server: threads should be started here; exceptions may be raised + * Start the provider: threads should be started here; exceptions may be raised * if the history provider cannot be started. - * @param historyBinding binding information + * The base implementation contains a re-entrancy check and should + * be invoked first. * @return the metric and binding information for registration */ - def start(historyBinding: ApplicationHistoryBinding): - (Option[Source], Option[HealthCheckSource]) = { - require(binding.isEmpty, "History provider already started") - binding = Some(historyBinding) + def start(): (Option[Source], Option[HealthCheckSource]) = { + require(started.getAndSet(true), "History provider already started") (None, None) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 0c3508ec80b74..b8f5e6678f959 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -175,14 +175,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Bind to the History Server: threads should be started here; exceptions may be raised + * Start the provider: threads should be started here; exceptions may be raised * if the history provider cannot be started. - * @param historyBinding binding information * @return the metric and binding information for registration */ - override def start(historyBinding: ApplicationHistoryBinding): - (Option[Source], Option[HealthCheckSource]) = { - super.start(historyBinding) + override def start(): (Option[Source], Option[HealthCheckSource]) = { + super.start() (Some(metrics), Some(health)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 7b7c1c422fef1..386983df407cd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -160,7 +160,7 @@ class HistoryServer( // hook up metrics // start the provider against the metrics binding - val (source, health) = provider.start(new ApplicationHistoryBinding()) + val (source, health) = provider.start() val codahaleContext = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) codahaleContext.addServlet(new ServletHolder(new ThreadDumpServlet()), HistoryServer.THREADS_SUBPATH) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 1b5899a37272a..61debba10ae44 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -79,8 +79,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc */ private def createHistoryProvider(): FsHistoryProvider = { val provider = new FsHistoryProvider(createTestConf()) - val binding = new ApplicationHistoryBinding() - provider.start(binding) + provider.start() provider } From 59a4a6764b4f3b7845327542fecd4b0e2c732739 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 9 Dec 2015 18:06:53 +0000 Subject: [PATCH 07/22] SPARK-11373 cut the health checks out --- core/pom.xml | 8 ---- .../history/ApplicationHistoryProvider.scala | 8 ++-- .../deploy/history/FsHistoryProvider.scala | 40 +++--------------- .../spark/deploy/history/HistoryMetrics.scala | 41 ------------------- .../spark/deploy/history/HistoryServer.scala | 31 +++++--------- .../history/FsHistoryProviderSuite.scala | 3 -- pom.xml | 10 ----- 7 files changed, 20 insertions(+), 121 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala diff --git a/core/pom.xml b/core/pom.xml index 1eaec171c93d5..97a463abbefdd 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -253,14 +253,6 @@ io.dropwizard.metrics metrics-graphite - - io.dropwizard.metrics - metrics-healthchecks - - - io.dropwizard.metrics - metrics-servlets - com.fasterxml.jackson.core jackson-databind diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index b43addd44420e..832883423a433 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -110,11 +110,11 @@ private[history] abstract class ApplicationHistoryProvider { * if the history provider cannot be started. * The base implementation contains a re-entrancy check and should * be invoked first. - * @return the metric and binding information for registration + * @return the metric information for registration */ - def start(): (Option[Source], Option[HealthCheckSource]) = { - require(started.getAndSet(true), "History provider already started") - (None, None) + def start(): Option[Source] = { + require(!started.getAndSet(true), "History provider already started") + None } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index b8f5e6678f959..fa6a4c7ebcedd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -22,8 +22,6 @@ import java.util.UUID import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} -import com.codahale.metrics.health.HealthCheck.Result -import com.codahale.metrics.health.{HealthCheck, HealthCheckRegistry} import com.codahale.metrics._ import org.apache.spark.metrics.source.Source @@ -136,9 +134,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** filesystem metrics: visible for test access */ private[history] val metrics = new FsHistoryProviderMetrics(this) - /** filesystem health: visible for test access */ - private[history] val health = new FsHistoryProviderHealth(this) - /** * Return a runnable that performs the given operation on the event logs. * This operation is expected to be executed periodically. @@ -177,11 +172,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** * Start the provider: threads should be started here; exceptions may be raised * if the history provider cannot be started. - * @return the metric and binding information for registration + * The base implementation contains a re-entrancy check and should + * be invoked first. + * @return the metric information for registration */ - override def start(): (Option[Source], Option[HealthCheckSource]) = { + override def start(): Option[Source] = { super.start() - (Some(metrics), Some(health)) + Some(metrics) } private[history] def startSafeModeCheckThread( @@ -798,33 +795,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Health checks - */ -private[history] class FsHistoryProviderHealth(owner: FsHistoryProvider) extends HealthCheckSource { - - override val sourceName = "history.fs" - - private val name = MetricRegistry.name(sourceName) - val healthRegistry = new HealthCheckRegistry - - val fileSystemLive = new HealthCheck { - override def check() = { - if (owner.isFsInSafeMode()) { - Result.unhealthy("Filesystem is in safe mode") - } else { - Result.healthy() - } - } - } - - private val healthChecks = Seq( - ("filesystem.healthy", fileSystemLive) - ) - - healthChecks.foreach(elt => healthRegistry.register(elt._1, elt._2)) - -} - /** * Metrics integration: the various counters of activity */ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extends Source { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala deleted file mode 100644 index 50ef12db474c6..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetrics.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.history - -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.health.HealthCheckRegistry - -import org.apache.spark.metrics.source.Source - -/** - * History system metrics independent of providers go in here - * @param owner owning instance - */ -private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { - override val metricRegistry = new MetricRegistry() - override val sourceName = "history" -} - -/** - * A trait for sources of health checks to implement - */ -private[spark] trait HealthCheckSource { - def sourceName: String - - def healthRegistry: HealthCheckRegistry -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 386983df407cd..75372b2b8f8a0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -26,16 +26,12 @@ import scala.util.control.NonFatal import scala.xml.Node import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.health.HealthCheckRegistry -import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} -//import com.codahale.metrics.jvm.{ThreadStatesGaugeSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} -import com.codahale.metrics.servlets.HealthCheckServlet -import com.codahale.metrics.servlets.ThreadDumpServlet import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.source.Source import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -76,7 +72,6 @@ class HistoryServer( private[history] val metricsSystem: MetricsSystem = MetricsSystem.createMetricsSystem("history", conf, securityManager) private[history] var metricsReg = metricsSystem.getMetricRegistry - private[history] var healthChecks: Option[HealthCheckSource] = _ private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { @@ -160,17 +155,7 @@ class HistoryServer( // hook up metrics // start the provider against the metrics binding - val (source, health) = provider.start() - val codahaleContext = new ServletContextHandler(null, HistoryServer.METRICS_PATH_PREFIX) - codahaleContext.addServlet(new ServletHolder(new ThreadDumpServlet()), - HistoryServer.THREADS_SUBPATH) - health.foreach { healthSource => - codahaleContext.addServlet( - new ServletHolder(new HealthCheckServlet(healthSource.healthRegistry)), - HistoryServer.HEALTH_SUBPATH) - } - healthChecks = health - attachHandler(codahaleContext) + val source = provider.start() source.foreach(metricsSystem.registerSource) metricsSystem.registerSource(new HistoryMetrics(this)) metricsSystem.start() @@ -306,6 +291,15 @@ class HistoryServer( } } +/** + * History system metrics independent of providers go in here + * @param owner owning instance + */ +private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { + override val metricRegistry = new MetricRegistry() + override val sourceName = "history" +} + /** * The recommended way of starting and stopping a HistoryServer is through the scripts * start-history-server.sh and stop-history-server.sh. The path to a base log directory, @@ -321,9 +315,6 @@ object HistoryServer extends Logging { private val conf = new SparkConf val UI_PATH_PREFIX = "/history" - val METRICS_PATH_PREFIX = "/metrics" - val HEALTH_SUBPATH = "/health" - val THREADS_SUBPATH = "/threads" def main(argStrings: Array[String]): Unit = { Utils.initDaemon(log) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 61debba10ae44..f738cc2ec68a4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -24,7 +24,6 @@ import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.health.HealthCheckRegistry import scala.concurrent.duration._ import scala.language.postfixOps @@ -49,12 +48,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc private var testDir: File = null private var metrics: MetricRegistry = _ - private var health: HealthCheckRegistry = _ before { testDir = Utils.createTempDir(namePrefix = s"a b%20c+d") metrics = new MetricRegistry() - health = new HealthCheckRegistry() } after { diff --git a/pom.xml b/pom.xml index c66f6358655ef..c1174593c1922 100644 --- a/pom.xml +++ b/pom.xml @@ -592,16 +592,6 @@ metrics-graphite ${codahale.metrics.version} - - io.dropwizard.metrics - metrics-healthchecks - ${codahale.metrics.version} - - - io.dropwizard.metrics - metrics-servlets - ${codahale.metrics.version} - com.fasterxml.jackson.core jackson-databind From 4c30404f2b2ddf1a7dc8b2de8d9c371716dd0c9e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 5 Feb 2016 14:43:00 +0000 Subject: [PATCH 08/22] [SPARK-11373] tail end of rebase operation --- .../spark/deploy/history/FsHistoryProviderSuite.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index f738cc2ec68a4..63d704cc7b108 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -145,13 +145,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } - private def createHistoryProvider(): FsHistoryProvider = { - val provider = new FsHistoryProvider(createTestConf()) - val binding = new ApplicationHistoryBinding(metrics, health) - provider.start(binding) - provider - } - test("SPARK-3697: ignore directories that cannot be read.") { // setReadable(...) does not work on Windows. Please refer JDK-6728842. assume(!Utils.isWindows) From ee74f817fc1ce969d67eb4d51f646e802cd0236c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 8 Feb 2016 12:37:40 +0000 Subject: [PATCH 09/22] [SPARK-11373] scalastyle and import ordering --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 5 ++--- .../org/apache/spark/deploy/history/HistoryServer.scala | 6 +++--- .../main/scala/org/apache/spark/metrics/MetricsSystem.scala | 2 +- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index fa6a4c7ebcedd..4f8802d0ac10f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -22,12 +22,10 @@ import java.util.UUID import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} -import com.codahale.metrics._ -import org.apache.spark.metrics.source.Source - import scala.collection.mutable import scala.xml.Node +import com.codahale.metrics._ import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -38,6 +36,7 @@ import org.apache.hadoop.security.AccessControlException import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.ui.SparkUI diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 75372b2b8f8a0..75b474512e3af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -29,12 +29,12 @@ import com.codahale.metrics.MetricRegistry import com.google.common.cache._ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.metrics.source.Source +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.source.Source import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index ab0d83c07902f..cda23b2001301 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import com.codahale.metrics.{MetricRegistry, Metric, MetricFilter} +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} From a3a63837a9dd420d1eeac75cd0546ebcd5f3f99e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 12 Feb 2016 13:18:40 +0000 Subject: [PATCH 10/22] SPARK-11373 finish review of merge with trunk; add new Timestamp gauge and set to time of update/update success. Makes update time visible for tests/users/ops --- .../deploy/history/FsHistoryProvider.scala | 51 +++++++++++-------- .../spark/deploy/history/HistoryServer.scala | 43 +++++++++------- 2 files changed, 56 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 4f8802d0ac10f..ff2387173eae4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -346,6 +346,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private[history] def checkForLogs(): Unit = { metrics.updateCount.inc() + metrics.updateLastAttempted.touch() time(metrics.updateTimer) { try { val newLastScanTime = getNewLastScanTime() @@ -418,6 +419,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } lastScanTime.set(newLastScanTime) + metrics.updateLastSucceeded.setValue(newLastScanTime) } catch { case e: Exception => logError( "Exception in checking for event log updates", @@ -487,6 +489,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + /** * Replay the log files in the list and merge the list of old applications with new ones */ @@ -778,15 +781,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** * Metrics integration: the various counters of activity * Time a closure, returning its output. - * @param t timer - * @param f function + * @param timer timer + * @param fn function * @tparam T type of return value of time * @return the result of the function. */ - private def time[T](t: Timer)(f: => T): T = { - val timeCtx = t.time() + private def time[T](timer: Timer)(fn: => T): T = { + val timeCtx = timer.time() try { - f + fn } finally { timeCtx.close() } @@ -794,7 +797,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Metrics integration: the various counters of activity + * Metrics integration: the various counters of activity. */ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extends Source { @@ -802,36 +805,44 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extend private val name = MetricRegistry.name(sourceName) - /** Metrics registry */ + /** Metrics registry. */ override val metricRegistry = new MetricRegistry() - /** Number of updates */ + /** Number of updates. */ val updateCount = new Counter() - /** Number of update failures */ + /** Number of update failures. */ val updateFailureCount = new Counter() - /** Number of App UI load operations */ + /** Update duration timer. */ + val updateTimer = new Timer() + + /** Time the last update was attempted. */ + val updateLastAttempted = new Timestamp() + + /** Time the last update succeded. */ + val updateLastSucceeded = new Timestamp() + + /** Number of App UI load operations. */ val appUILoadCount = new Counter() - /** Number of App UI load operations that failed */ + /** Number of App UI load operations that failed due to a load/parse/replay problem. */ val appUILoadFailureCount = new Counter() - /** Number of App UI load operations that failed due to an unknown file */ + /** Number of App UI load operations that failed due to an unknown file. */ val appUILoadNotFoundCount = new Counter() - /** Statistics on update duration */ - val updateTimer = new Timer() - - /** Statistics on time to load app UIs */ + /** Statistics on time to load app UIs. */ val appUiLoadTimer = new Timer() - /** Statistics on time to replay and merge listings */ + /** Statistics on time to replay and merge listings. */ val mergeApplicationListingTimer = new Timer() - private val counters = Seq( + private val countersAndGauges = Seq( ("update.count", updateCount), ("update.failure.count", updateFailureCount), + ("update.last.attempted", updateLastAttempted), + ("update.last.succeeded", updateLastSucceeded), ("appui.load.count", appUILoadCount), ("appui.load.failure.count", appUILoadFailureCount)) ("appui.load.not-found.count", appUILoadNotFoundCount) @@ -841,12 +852,12 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extend ("merge.application.listings.timer", mergeApplicationListingTimer), ("appui.load.timer", appUiLoadTimer)) - val allMetrics = counters ++ timers + val allMetrics = countersAndGauges ++ timers allMetrics.foreach(elt => metricRegistry.register(elt._1, elt._2)) override def toString: String = { - def sb = new StringBuilder(counters.size * 20) + def sb = new StringBuilder(countersAndGauges.size * 20) allMetrics.foreach(elt => sb.append(s" ${elt._1} = ${elt._2}\n")) sb.toString() } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 75b474512e3af..5861f46530400 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,8 +25,7 @@ import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.util.control.NonFatal import scala.xml.Node -import com.codahale.metrics.MetricRegistry -import com.google.common.cache._ +import com.codahale.metrics.{Gauge, MetricRegistry} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} @@ -69,22 +68,10 @@ class HistoryServer( private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) private val initialized = new AtomicBoolean(false) - private[history] val metricsSystem: MetricsSystem = MetricsSystem.createMetricsSystem("history", + private[history] val metricsSystem = MetricsSystem.createMetricsSystem("history", conf, securityManager) private[history] var metricsReg = metricsSystem.getMetricRegistry - private val appLoader = new CacheLoader[String, SparkUI] { - override def load(key: String): SparkUI = { - val parts = key.split("/") - require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") - val ui = provider - .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) - .getOrElse(throw new NoSuchElementException(s"no app with key $key")) - attachSparkUI(ui) - ui - } - } - // and its metrics, for testing as well as monitoring val cacheMetrics = appCache.metrics @@ -154,10 +141,9 @@ class HistoryServer( // hook up metrics - // start the provider against the metrics binding - val source = provider.start() - source.foreach(metricsSystem.registerSource) + provider.start().foreach(metricsSystem.registerSource) metricsSystem.registerSource(new HistoryMetrics(this)) + metricsSystem.registerSource(appCache.metrics) metricsSystem.start() metricsSystem.getServletHandlers.foreach(attachHandler) } @@ -300,6 +286,27 @@ private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { override val sourceName = "history" } +/** + * A timestamp is a gauge which is set to a point in time + * as measured in millseconds since the epoch began. + */ +private[history] class Timestamp extends Gauge[Long] { + var time = 0L + + /** Current value. */ + override def getValue: Long = time + + /** set a new value. */ + def setValue(t: Long): Unit = { + time = t + } + + /** Set to the current system time. */ + def touch(): Unit = { + setValue(System.currentTimeMillis()) + } +} + /** * The recommended way of starting and stopping a HistoryServer is through the scripts * start-history-server.sh and stop-history-server.sh. The path to a base log directory, From e936d5bcfd53a4694fd1d396d656e3c1e5926773 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 26 Apr 2016 18:50:44 +0100 Subject: [PATCH 11/22] [SPARK-11373] finish rebasing to master, correct tightened style checks --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- .../org/apache/spark/deploy/history/HistoryServer.scala | 6 ++---- .../spark/deploy/history/FsHistoryProviderSuite.scala | 3 +-- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index ff2387173eae4..42702a0966810 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -25,7 +25,7 @@ import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable import scala.xml.Node -import com.codahale.metrics._ +import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 5861f46530400..c668b138185aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -124,7 +124,8 @@ class HistoryServer( /** * Initialize the history server. * - * This calls [[ApplicationHistoryProvider.start()]] to start the history provider. + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. */ def initialize() { if (!initialized.getAndSet(true)) { @@ -140,7 +141,6 @@ class HistoryServer( attachHandler(contextHandler) // hook up metrics - provider.start().foreach(metricsSystem.registerSource) metricsSystem.registerSource(new HistoryMetrics(this)) metricsSystem.registerSource(appCache.metrics) @@ -156,8 +156,6 @@ class HistoryServer( /** Stop the server and close the history provider. */ override def stop() { - super.stop() - provider.stop() try { super.stop() } finally { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 63d704cc7b108..00e80f6a04ed8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -23,11 +23,10 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} -import com.codahale.metrics.MetricRegistry - import scala.concurrent.duration._ import scala.language.postfixOps +import com.codahale.metrics.MetricRegistry import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.hdfs.DistributedFileSystem import org.json4s.jackson.JsonMethods._ From cef15774fd421ed13bc830dbd647b40bac2caad3 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 10 Jun 2016 15:10:46 +0100 Subject: [PATCH 12/22] [SPARK-11373] Address review comments and add a new counter of events played back during merge and U load --- .../history/ApplicationHistoryProvider.scala | 85 +++++++++++++++++++ .../deploy/history/FsHistoryProvider.scala | 25 ++++-- .../deploy/history/HistoryServerSuite.scala | 1 - 3 files changed, 105 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 832883423a433..6616292a76195 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -26,6 +26,7 @@ import com.codahale.metrics.health.HealthCheckRegistry import org.apache.spark.SparkException import org.apache.spark.metrics.source.Source +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerBlockManagerRemoved, SparkListenerBlockUpdated, SparkListenerEnvironmentUpdate, SparkListenerEvent, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskGettingResult, SparkListenerTaskStart, SparkListenerUnpersistRDD} import org.apache.spark.ui.SparkUI private[spark] case class ApplicationAttemptInfo( @@ -164,3 +165,87 @@ private[history] abstract class ApplicationHistoryProvider { */ def getEmptyListingHtml(): Seq[Node] = Seq.empty } + +/** + * A simple counter of events. + * There is no concurrency support here: all events must come in sequentially. + */ +private[history] class EventCountListener extends SparkListener { + var eventCount = 0L + + def process(event: SparkListenerEvent): Unit = { + eventCount += 1 + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + process(stageCompleted) + } + + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { + process(event) + } + + override def onTaskStart(event: SparkListenerTaskStart): Unit = { + process(event) + } + + override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = { + process(event) + } + + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { + process(event) + } + + override def onJobStart(event: SparkListenerJobStart): Unit = { + process(event) + } + + override def onJobEnd(event: SparkListenerJobEnd): Unit = { + process(event) + } + + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { + process(event) + } + + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = { + process(event) + } + + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { + process(event) + } + + override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { + process(event) + } + + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { + process(event) + } + + override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { + process(event) + } + + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + process(event) + } + + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { + process(event) + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { + process(event) + } + + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { + process(event) + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + process(event) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 42702a0966810..817808141729e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -28,7 +28,7 @@ import scala.xml.Node import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException @@ -282,7 +282,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) - val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) + val (appListener, count) = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) + metrics.eventReplayedCount.inc(count) if (appListener.appId.isDefined) { ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) @@ -508,7 +509,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // won't change whenever HistoryServer restarts and reloads the file. val lastUpdated = if (appCompleted) fileStatus.getModificationTime else clock.getTimeMillis() - val appListener = replay(fileStatus, appCompleted, new ReplayListenerBus(), eventsFilter) + val (appListener, count) = replay(fileStatus, appCompleted, new ReplayListenerBus(), eventsFilter) + metrics.eventReplayedCount.inc(count) // Without an app ID, new logs will render incorrectly in the listing page, so do not list or // try to show their UI. @@ -677,12 +679,19 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * an `ApplicationEventListener` instance with event data captured from the replay. * `ReplayEventsFilter` determines what events are replayed and can therefore limit the * data captured in the returned `ApplicationEventListener` instance. + * + * @param eventLog reference to the event log to play back. + * @param appCompleted has the application completed? + * @param bus event bus to play events to + * @param eventsFilter filter for events + * @return the event listener and the number of processed events + * */ private def replay( eventLog: FileStatus, appCompleted: Boolean, bus: ReplayListenerBus, - eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): ApplicationEventListener = { + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): (ApplicationEventListener, Long) = { val logPath = eventLog.getPath() logInfo(s"Replaying log path: $logPath") // Note that the eventLog may have *increased* in size since when we grabbed the filestatus, @@ -693,10 +702,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // after it's created, so we get a file size that is no bigger than what is actually read. val logInput = EventLoggingListener.openEventLog(logPath, fs) try { + var countListener = new EventCountListener() val appListener = new ApplicationEventListener bus.addListener(appListener) + bus.addListener(countListener) bus.replay(logInput, logPath.toString, !appCompleted, eventsFilter) - appListener + (appListener, countListener.eventCount) } finally { logInput.close() } @@ -814,6 +825,9 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extend /** Number of update failures. */ val updateFailureCount = new Counter() + /** Number of events replayed. This includes those on UI playback as well as listing merge. */ + val eventReplayedCount = new Counter() + /** Update duration timer. */ val updateTimer = new Timer() @@ -839,6 +853,7 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extend val mergeApplicationListingTimer = new Timer() private val countersAndGauges = Seq( + ("event.replay.count", eventReplayedCount), ("update.count", updateCount), ("update.failure.count", updateFailureCount), ("update.last.attempted", updateLastAttempted), diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 3cc98d397f5c2..f715d4cd1fc5f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -416,7 +416,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // stop the server with the old config, and start the new one server.stop() server = new HistoryServer(myConf, provider, securityManager, 18080) - server.initialize() server.bind() val port = server.boundPort val metrics = server.cacheMetrics From 4bf9d13fc5ab0dff7d0cc340e62d466e7da127db Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 14 Jul 2016 14:29:38 +0100 Subject: [PATCH 13/22] [SPARK-11373] more metrics, all sources have prefixes, common features pulled up into HistoryMetricSource, including registration and prefix support. The existing test case "incomplete apps get refreshed" has been extended to check for metrics in the listings, alongside some specific probes for values in the fs history provider (loads, load time average > 0) --- .../deploy/history/ApplicationCache.scala | 24 +--- .../history/ApplicationHistoryProvider.scala | 82 +----------- .../deploy/history/FsHistoryProvider.scala | 88 ++++++++----- .../spark/deploy/history/HistoryServer.scala | 122 ++++++++++++++++-- .../deploy/history/HistoryServerSuite.scala | 56 ++++++-- 5 files changed, 230 insertions(+), 142 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index a370526c46f3d..6a44ecd029d13 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -24,12 +24,11 @@ import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import com.codahale.metrics.{Counter, MetricRegistry, Timer} +import com.codahale.metrics.{Counter, Counting, Metric, MetricRegistry, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} import org.eclipse.jetty.servlet.FilterHolder import org.apache.spark.internal.Logging -import org.apache.spark.metrics.source.Source import org.apache.spark.ui.SparkUI import org.apache.spark.util.Clock @@ -388,7 +387,7 @@ private[history] final case class CacheKey(appId: String, attemptId: Option[Stri * Metrics of the cache * @param prefix prefix to register all entries under */ -private[history] class CacheMetrics(prefix: String) extends Source { +private[history] class CacheMetrics(prefix: String) extends HistoryMetricSource(prefix) { /* metrics: counters and timers */ val lookupCount = new Counter() @@ -410,34 +409,25 @@ private[history] class CacheMetrics(prefix: String) extends Source { ("update.triggered.count", updateTriggeredCount)) /** all metrics, including timers */ - private val allMetrics = counters ++ Seq( + private val allMetrics: Seq[(String, Metric with Counting)] = counters ++ Seq( ("load.timer", loadTimer), ("update.probe.timer", updateProbeTimer)) /** * Name of metric source */ - override val sourceName = "ApplicationCache" - - override val metricRegistry: MetricRegistry = new MetricRegistry + override val sourceName = "application.cache" /** * Startup actions. * This includes registering metrics with [[metricRegistry]] */ private def init(): Unit = { - allMetrics.foreach { case (name, metric) => - metricRegistry.register(MetricRegistry.name(prefix, name), metric) - } + register(allMetrics) } - override def toString: String = { - val sb = new StringBuilder() - counters.foreach { case (name, counter) => - sb.append(name).append(" = ").append(counter.getCount).append('\n') - } - sb.toString() - } + init() + } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 6616292a76195..ab7ec930603dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -21,12 +21,10 @@ import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.ZipOutputStream import scala.xml.Node -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.health.HealthCheckRegistry -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, SparkFirehoseListener} import org.apache.spark.metrics.source.Source -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerBlockManagerRemoved, SparkListenerBlockUpdated, SparkListenerEnvironmentUpdate, SparkListenerEvent, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskGettingResult, SparkListenerTaskStart, SparkListenerUnpersistRDD} +import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI private[spark] case class ApplicationAttemptInfo( @@ -170,82 +168,10 @@ private[history] abstract class ApplicationHistoryProvider { * A simple counter of events. * There is no concurrency support here: all events must come in sequentially. */ -private[history] class EventCountListener extends SparkListener { +private[history] class EventCountListener extends SparkFirehoseListener { var eventCount = 0L - def process(event: SparkListenerEvent): Unit = { + override def onEvent(event: SparkListenerEvent): Unit = { eventCount += 1 } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - process(stageCompleted) - } - - override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { - process(event) - } - - override def onTaskStart(event: SparkListenerTaskStart): Unit = { - process(event) - } - - override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = { - process(event) - } - - override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { - process(event) - } - - override def onJobStart(event: SparkListenerJobStart): Unit = { - process(event) - } - - override def onJobEnd(event: SparkListenerJobEnd): Unit = { - process(event) - } - - override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { - process(event) - } - - override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = { - process(event) - } - - override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { - process(event) - } - - override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { - process(event) - } - - override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { - process(event) - } - - override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { - process(event) - } - - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - process(event) - } - - override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { - process(event) - } - - override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { - process(event) - } - - override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { - process(event) - } - - override def onOtherEvent(event: SparkListenerEvent): Unit = { - process(event) - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 817808141729e..88e5c763c0366 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -131,7 +131,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val pendingReplayTasksCount = new java.util.concurrent.atomic.AtomicInteger(0) /** filesystem metrics: visible for test access */ - private[history] val metrics = new FsHistoryProviderMetrics(this) + private[history] val metrics = new FsHistoryProviderMetrics(this, "history.provider") /** * Return a runnable that performs the given operation on the event logs. @@ -266,8 +266,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { - metrics.appUILoadCount.inc() - time(metrics.appUiLoadTimer) { + metrics.appUILoadCount.inc() + time(metrics.appUiLoadTimer, Some(metrics.appUITotalLoadTime)) { try { applications.get(appId).flatMap { appInfo => appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt => @@ -283,7 +283,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) val (appListener, count) = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) - metrics.eventReplayedCount.inc(count) + metrics.appUIEventCount.inc(count) if (appListener.appId.isDefined) { ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) @@ -387,8 +387,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { for (file <- logInfos) { tasks += replayExecutor.submit(new Runnable { - override def run(): Unit = time(metrics.mergeApplicationListingTimer) { - mergeApplicationListing(file) + override def run(): Unit = + time(metrics.historyMergeTimer, Some(metrics.historyTotalMergeTime)) { + mergeApplicationListing(file) } }) } @@ -510,7 +511,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val lastUpdated = if (appCompleted) fileStatus.getModificationTime else clock.getTimeMillis() val (appListener, count) = replay(fileStatus, appCompleted, new ReplayListenerBus(), eventsFilter) - metrics.eventReplayedCount.inc(count) + metrics.historyEventCount.inc(count) // Without an app ID, new logs will render incorrectly in the listing page, so do not list or // try to show their UI. @@ -620,7 +621,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) appsToRetain += (app.id -> app) } else if (toRetain.nonEmpty) { appsToRetain += (app.id -> - new FsApplicationHistoryInfo(app.id, app.name, toRetain.toList)) + new FsApplicationHistoryInfo(app.id, app.name, toRetain)) } } @@ -790,19 +791,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Metrics integration: the various counters of activity * Time a closure, returning its output. + * The timer is updated with the duration, and if a counter is supplied, it's count + * is incremented by the duration. * @param timer timer + * @param counter counter: an optional counter of the duration * @param fn function * @tparam T type of return value of time * @return the result of the function. */ - private def time[T](timer: Timer)(fn: => T): T = { + private def time[T](timer: Timer, counter: Option[Counter] = None)(fn: => T): T = { val timeCtx = timer.time() try { fn } finally { - timeCtx.close() + val duration = timeCtx.stop() + counter.foreach(_.inc(duration)) } } } @@ -810,23 +814,44 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) /** * Metrics integration: the various counters of activity. */ -private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extends Source { +private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix: String) + extends HistoryMetricSource(prefix) { + + /** + * Function to return an average; if the count is 0, so is the average. + * @param value value to average + * @param count event count to divide by + * @return the average, or 0 if the counter is itself 0 + */ + private def average(value: Long, count: Long): Long = { + if (count> 0) value / count else 0 + } override val sourceName = "history.fs" private val name = MetricRegistry.name(sourceName) - /** Metrics registry. */ - override val metricRegistry = new MetricRegistry() - /** Number of updates. */ val updateCount = new Counter() /** Number of update failures. */ val updateFailureCount = new Counter() - /** Number of events replayed. This includes those on UI playback as well as listing merge. */ - val eventReplayedCount = new Counter() + /** Number of events replayed as listing merge. */ + val historyEventCount = new Counter() + + /** Timer of listing merges. */ + val historyMergeTimer = new Timer() + + /** Total time to merge all histories. */ + val historyTotalMergeTime = new Counter() + + /** Average time to load a single event in the App UI */ + val historyEventMergeTime = new LambdaLongGauge(() => + average(historyTotalMergeTime.getCount, historyEventCount.getCount)) + + /** Number of events replayed. */ + val appUIEventCount = new Counter() /** Update duration timer. */ val updateTimer = new Timer() @@ -849,33 +874,38 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider) extend /** Statistics on time to load app UIs. */ val appUiLoadTimer = new Timer() - /** Statistics on time to replay and merge listings. */ - val mergeApplicationListingTimer = new Timer() + /** Total load time of all App UIs. */ + val appUITotalLoadTime = new Counter() + + /** Average time to load a single event in the App UI */ + val appUIEventReplayTime = new LambdaLongGauge(() => + average(appUITotalLoadTime.getCount, appUIEventCount.getCount)) private val countersAndGauges = Seq( - ("event.replay.count", eventReplayedCount), + ("history.merge.event.count", historyEventCount), + ("history.merge.event.time", historyEventMergeTime), + ("history.merge.duration", historyTotalMergeTime), ("update.count", updateCount), ("update.failure.count", updateFailureCount), ("update.last.attempted", updateLastAttempted), ("update.last.succeeded", updateLastSucceeded), ("appui.load.count", appUILoadCount), - ("appui.load.failure.count", appUILoadFailureCount)) - ("appui.load.not-found.count", appUILoadNotFoundCount) + ("appui.load.duration", appUITotalLoadTime), + ("appui.load.failure.count", appUILoadFailureCount), + ("appui.load.not-found.count", appUILoadNotFoundCount), + ("appui.event.count", appUIEventCount), + ("appui.event.replay.time", appUIEventReplayTime) + ) private val timers = Seq ( ("update.timer", updateTimer), - ("merge.application.listings.timer", mergeApplicationListingTimer), + ("history.merge.timer", historyMergeTimer), ("appui.load.timer", appUiLoadTimer)) val allMetrics = countersAndGauges ++ timers - allMetrics.foreach(elt => metricRegistry.register(elt._1, elt._2)) + register(allMetrics) - override def toString: String = { - def sb = new StringBuilder(countersAndGauges.size * 20) - allMetrics.foreach(elt => sb.append(s" ${elt._1} = ${elt._2}\n")) - sb.toString() - } } private[history] object FsHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index c668b138185aa..44c4c68b6c01a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -22,10 +22,11 @@ import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import scala.xml.Node -import com.codahale.metrics.{Gauge, MetricRegistry} +import com.codahale.metrics.{Counter, Counting, Gauge, Metric, MetricFilter, MetricRegistry, Timer} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} @@ -70,7 +71,7 @@ class HistoryServer( private[history] val metricsSystem = MetricsSystem.createMetricsSystem("history", conf, securityManager) - private[history] var metricsReg = metricsSystem.getMetricRegistry + private[history] var metricsRegistry = metricsSystem.getMetricRegistry // and its metrics, for testing as well as monitoring val cacheMetrics = appCache.metrics @@ -119,6 +120,11 @@ class HistoryServer( appCache.getSparkUI(appKey) } + val historyMetrics = new HistoryMetrics(this, "history.server") + // provider metrics are None until the provider is started, and only after that + // point if the provider returns any. + var providerMetrics: Option[Source] = None + initialize() /** @@ -141,9 +147,10 @@ class HistoryServer( attachHandler(contextHandler) // hook up metrics - provider.start().foreach(metricsSystem.registerSource) - metricsSystem.registerSource(new HistoryMetrics(this)) + metricsSystem.registerSource(historyMetrics) metricsSystem.registerSource(appCache.metrics) + providerMetrics = provider.start() + providerMetrics.foreach(metricsSystem.registerSource) metricsSystem.start() metricsSystem.getServletHandlers.foreach(attachHandler) } @@ -276,12 +283,102 @@ class HistoryServer( } /** - * History system metrics independent of providers go in here - * @param owner owning instance + * An abstract implementation of the metrics [[Source]] trait with some common operations. */ -private[history] class HistoryMetrics(val owner: HistoryServer) extends Source { +private[history] abstract class HistoryMetricSource(val prefix: String) extends Source { override val metricRegistry = new MetricRegistry() + + /** + * Register a sequence of metrics + * @param metrics sequence of metrics to register + */ + def register(metrics: Seq[(String, Metric)]): Unit = { + metrics.foreach { case (name, metric) => + metricRegistry.register(fullname(name), metric) + } + } + + /** + * Create the full name of a metric by prepending the prefix to the name + * @param name short name + * @return the full name to use in registration + */ + def fullname(name: String): String = { + MetricRegistry.name(prefix, name) + } + + /** + * Dump the counters and gauges. + * @return a string for logging and diagnostics —not for parsing by machines. + */ + override def toString: String = { + val sb = new StringBuilder(s"Metrics for $sourceName:\n") + sb.append(" Counters\n") + metricRegistry.getCounters.asScala.foreach { entry => + sb.append(" ").append(entry._1).append(" = ").append(entry._2.getCount).append('\n') + } + sb.append(" Gauges\n") + metricRegistry.getGauges.asScala.foreach { entry => + sb.append(" ").append(entry._1).append(" = ").append(entry._2.getValue).append('\n') + } + sb.toString() + } + + /** + * Get a named counter. + * @param counterName name of the counter + * @return the counter, if found + */ + def getCounter(counterName: String): Option[Counter] = { + Option(metricRegistry.getCounters(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name == counterName + }).get(counterName)) + } + + /** + * Get a gauge of an unknown numeric type. + * @param gaugeName name of the gauge + * @return gauge, if found + */ + def getGauge(gaugeName: String): Option[Gauge[_]] = { + Option(metricRegistry.getGauges(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name == gaugeName + }).get(gaugeName)) + } + + /** + * Get a Long gauge. + * @param gaugeName name of the gauge + * @return gauge, if found + * @throws ClassCastException if the gauge is found but of the wrong type + */ + def getLongGauge(gaugeName: String): Option[Gauge[Long]] = { + Option(metricRegistry.getGauges(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name == gaugeName + }).get(gaugeName)).asInstanceOf[Option[Gauge[Long]]] + } + + /** + * Get a timer. + * @param timerName name of the timer + * @return the timer, if found. + */ + def getTimer(timerName: String): Option[Timer] = { + Option(metricRegistry.getTimers(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name == timerName + }).get(timerName)) + } + +} + +/** + * History system metrics independent of providers go in here. + * @param owner owning instance + */ +private[history] class HistoryMetrics(val owner: HistoryServer, prefix: String) + extends HistoryMetricSource(prefix) { override val sourceName = "history" + } /** @@ -294,7 +391,7 @@ private[history] class Timestamp extends Gauge[Long] { /** Current value. */ override def getValue: Long = time - /** set a new value. */ + /** Set a new value. */ def setValue(t: Long): Unit = { time = t } @@ -305,6 +402,15 @@ private[history] class Timestamp extends Gauge[Long] { } } +/** + * A Long gauge from a lambda expression; the expression is evaluated + * whenever the metrics are queried + * @param expression expression which generates the value. + */ +private[history] class LambdaLongGauge(expression: (() => Long)) extends Gauge[Long] { + override def getValue: Long = expression() +} + /** * The recommended way of starting and stopping a HistoryServer is through the scripts * start-history-server.sh and stop-history-server.sh. The path to a base log directory, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index f715d4cd1fc5f..6bbdf809b523e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.history -import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} +import java.io.{File, FileInputStream, FileWriter, IOException, InputStream} import java.net.{HttpURLConnection, URL} import java.nio.charset.StandardCharsets import java.util.zip.ZipInputStream @@ -26,7 +26,7 @@ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpSe import scala.concurrent.duration._ import scala.language.postfixOps -import com.codahale.metrics.Counter +import com.codahale.metrics.{Counter, Gauge} import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -43,6 +43,7 @@ import org.scalatest.mock.MockitoSugar import org.scalatest.selenium.WebBrowser import org.apache.spark._ +import org.apache.spark.metrics.source.Source import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.JobUIData import org.apache.spark.util.{ResetSystemProperties, Utils} @@ -418,15 +419,33 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers server = new HistoryServer(myConf, provider, securityManager, 18080) server.bind() val port = server.boundPort - val metrics = server.cacheMetrics + val cacheMetrics = server.cacheMetrics + val historyMetrics = server.historyMetrics + val providerMetrics = server.providerMetrics.get.asInstanceOf[HistoryMetricSource] + + // assert that a metric counter evaluates as expected; if not dump the whole metrics instance + def assertCounterEvaluates( + source: Source, + name: String, + counter: Counter, + evaluation: (Long => Boolean)): Unit = { + val value = counter.getCount + if (!evaluation(value)) { + // this is here because Scalatest loses stack depth + fail(s"Wrong $name value: $value in metrics\n$source") + } + } - // assert that a metric has a value; if not dump the whole metrics instance - def assertMetric(name: String, counter: Counter, expected: Long): Unit = { - val actual = counter.getCount - if (actual != expected) { + // assert that a long metric gauge evaluates as expected; if not dump the whole metrics instance + def assertGaugeEvaluates( + source: Source, + name: String, + gauge: Gauge[Long], + evaluation: (Long => Boolean)): Unit = { + val value = gauge.getValue + if (!evaluation(value)) { // this is here because Scalatest loses stack depth - fail(s"Wrong $name value - expected $expected but got $actual" + - s" in metrics\n$metrics") + fail(s"Wrong $name value: $value in metrics\n$source") } } @@ -515,7 +534,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers getNumJobs("") should be (1) getNumJobs("/jobs") should be (1) getNumJobsRestful() should be (1) - assert(metrics.lookupCount.getCount > 1, s"lookup count too low in $metrics") + assertCounterEvaluates(cacheMetrics, "lookup count", cacheMetrics.lookupCount, _ > 1) // dump state before the next bit of test, which is where update // checking really gets stressed @@ -563,6 +582,23 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers assert(jobcount === getNumJobs("/jobs")) + // print out the metrics. This forces a run through all the counters and gauges + // the evaluation is done outside the log statement to guarantee that the evaluation + // always takes place + val metricsDump = s"$historyMetrics\n$providerMetrics\n$cacheMetrics" + logInfo(s"Metrics:\n$metricsDump") + // make some assertions about internal state of providers via the metrics + val loadcount = providerMetrics.fullname("appui.load.count") + assert(metricsDump.contains(loadcount), s"No $loadcount in metrics dump <$metricsDump>") + assertCounterEvaluates(providerMetrics, loadcount, + providerMetrics.getCounter(loadcount).get, _ > 0) + val replayTime = providerMetrics.fullname("appui.event.replay.time") + assertGaugeEvaluates(providerMetrics, replayTime, + providerMetrics.getLongGauge(replayTime).get, _ > 0) + + val evictionCount = cacheMetrics.fullname("eviction.count") + assert(metricsDump.contains(evictionCount), s"No $evictionCount in metrics dump <$metricsDump>") + // no need to retain the test dir now the tests complete logDir.deleteOnExit(); From 818e14bd6ffb319dfadcb7275311467d027a95cd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 14 Jul 2016 14:35:20 +0100 Subject: [PATCH 14/22] [SPARK-11373] add a check that before there's been any events loaded, the average load time is "0" and not a division by zero error. This validates the logic in the relevant lambda-expression --- .../apache/spark/deploy/history/HistoryServerSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 6bbdf809b523e..e84ac7c3a8639 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -459,7 +459,10 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers new URL(s"http://localhost:$port/api/v1/applications/$appId$suffix") } - val historyServerRoot = new URL(s"http://localhost:$port/") + // check that a dynamically calculated average returns 0 and not a division by zero error. + val replayTime = providerMetrics.fullname("appui.event.replay.time") + assertGaugeEvaluates(providerMetrics, replayTime, + providerMetrics.getLongGauge(replayTime).get, _ == 0) // start initial job val d = sc.parallelize(1 to 10) @@ -592,7 +595,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers assert(metricsDump.contains(loadcount), s"No $loadcount in metrics dump <$metricsDump>") assertCounterEvaluates(providerMetrics, loadcount, providerMetrics.getCounter(loadcount).get, _ > 0) - val replayTime = providerMetrics.fullname("appui.event.replay.time") assertGaugeEvaluates(providerMetrics, replayTime, providerMetrics.getLongGauge(replayTime).get, _ > 0) From bfea17f9fd70bb3485b9dbf9ca8645735e518694 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 14 Jul 2016 14:48:50 +0100 Subject: [PATCH 15/22] [SPARK-11373] style check in the javadocs --- .../scala/org/apache/spark/deploy/history/HistoryServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 44c4c68b6c01a..0eecf29e761b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -309,7 +309,7 @@ private[history] abstract class HistoryMetricSource(val prefix: String) extends /** * Dump the counters and gauges. - * @return a string for logging and diagnostics —not for parsing by machines. + * @return a string for logging and diagnostics -not for parsing by machines. */ override def toString: String = { val sb = new StringBuilder(s"Metrics for $sourceName:\n") From b01facddabaefd9f525927ce211c98accb26d047 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 14 Jul 2016 14:57:18 +0100 Subject: [PATCH 16/22] [SPARK-11373] IDE had mysteriously re-ordered imports in the same line. Interesting, and not in a good way --- .../org/apache/spark/deploy/history/HistoryServerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index e84ac7c3a8639..29a4c1d18fa6e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.history -import java.io.{File, FileInputStream, FileWriter, IOException, InputStream} +import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} import java.nio.charset.StandardCharsets import java.util.zip.ZipInputStream From a956243e29d67e898c8236f7309a67eed2c32051 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 16 Aug 2016 19:50:19 +0100 Subject: [PATCH 17/22] [SPARK-11373] address latest comments -nits -recommended minor changes +pull out lambda gauge and HistoryMetricSource into their own file, HistoryMetricSource.scala. Added tests to go with this, and made the toString call robust against failing gauges +fixed a scaladoc warning in HistoryServer --- .../deploy/history/FsHistoryProvider.scala | 21 +- .../deploy/history/HistoryMetricSource.scala | 167 +++++++++++++++ .../spark/deploy/history/HistoryServer.scala | 197 +++++------------- .../apache/spark/metrics/MetricsSystem.scala | 2 +- .../history/HistoryMetricSourceSuite.scala | 89 ++++++++ .../deploy/history/HistoryServerSuite.scala | 4 +- 6 files changed, 315 insertions(+), 165 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/HistoryMetricSourceSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 88e5c763c0366..ee2c7133dc105 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -846,7 +846,7 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix /** Total time to merge all histories. */ val historyTotalMergeTime = new Counter() - /** Average time to load a single event in the App UI */ + /** Average time to process an event in the history merge operation. */ val historyEventMergeTime = new LambdaLongGauge(() => average(historyTotalMergeTime.getCount, historyEventCount.getCount)) @@ -856,11 +856,13 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix /** Update duration timer. */ val updateTimer = new Timer() + private val clock = new SystemClock + /** Time the last update was attempted. */ - val updateLastAttempted = new Timestamp() + val updateLastAttempted = new TimestampGauge(clock) /** Time the last update succeded. */ - val updateLastSucceeded = new Timestamp() + val updateLastSucceeded = new TimestampGauge(clock) /** Number of App UI load operations. */ val appUILoadCount = new Counter() @@ -881,7 +883,7 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix val appUIEventReplayTime = new LambdaLongGauge(() => average(appUITotalLoadTime.getCount, appUIEventCount.getCount)) - private val countersAndGauges = Seq( + register(Seq( ("history.merge.event.count", historyEventCount), ("history.merge.event.time", historyEventMergeTime), ("history.merge.duration", historyTotalMergeTime), @@ -894,17 +896,10 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix ("appui.load.failure.count", appUILoadFailureCount), ("appui.load.not-found.count", appUILoadNotFoundCount), ("appui.event.count", appUIEventCount), - ("appui.event.replay.time", appUIEventReplayTime) - ) - - private val timers = Seq ( + ("appui.event.replay.time", appUIEventReplayTime), ("update.timer", updateTimer), ("history.merge.timer", historyMergeTimer), - ("appui.load.timer", appUiLoadTimer)) - - val allMetrics = countersAndGauges ++ timers - - register(allMetrics) + ("appui.load.timer", appUiLoadTimer))) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala new file mode 100644 index 0000000000000..cada33d917d0c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import scala.collection.JavaConverters._ + +import com.codahale.metrics.{Counter, Gauge, Metric, MetricFilter, MetricRegistry, Timer} + +import org.apache.spark.metrics.source.Source +import org.apache.spark.util.Clock + +/** + * An abstract implementation of the metrics [[Source]] trait with some common operations for + * retrieving entries; the `toString()` operation dumps all counters and gauges. + */ +private[history] abstract class HistoryMetricSource(val prefix: String) extends Source { + override val metricRegistry = new MetricRegistry() + + /** + * Register a sequence of metrics + * @param metrics sequence of metrics to register + */ + def register(metrics: Seq[(String, Metric)]): Unit = { + metrics.foreach { case (name, metric) => + metricRegistry.register(fullname(name), metric) + } + } + + /** + * Create the full name of a metric by prepending the prefix to the name + * @param name short name + * @return the full name to use in registration + */ + def fullname(name: String): String = { + MetricRegistry.name(prefix, name) + } + + /** + * Dump the counters and gauges. + * @return a string for logging and diagnostics -not for parsing by machines. + */ + override def toString: String = { + val sb = new StringBuilder(s"Metrics for $sourceName:\n") + def robustAppend(s : => Long) = { + try { + sb.append(s) + } catch { + case e: Exception => + sb.append(s"(exception: $e)") + } + } + + sb.append(" Counters\n") + + metricRegistry.getCounters.asScala.foreach { case (name, counter) => + sb.append(" ").append(name).append(" = ") + .append(counter.getCount).append('\n') + } + sb.append(" Gauges\n") + metricRegistry.getGauges.asScala.foreach { case (name, gauge) => + sb.append(" ").append(name).append(" = ") + try { + sb.append(gauge.getValue) + } catch { + case e: Exception => + sb.append(s"(exception: $e)") + } + sb.append('\n') + } + sb.toString() + } + + /** + * Get a named counter. + * @param counterName name of the counter + * @return the counter, if found + */ + def getCounter(counterName: String): Option[Counter] = { + val full = fullname(counterName) + Option(metricRegistry.getCounters(new MetricByName(full)).get(full)) + } + + /** + * Get a gauge of an unknown numeric type. + * @param gaugeName name of the gauge + * @return gauge, if found + */ + def getGauge(gaugeName: String): Option[Gauge[_]] = { + val full = fullname(gaugeName) + Option(metricRegistry.getGauges(new MetricByName(full)).get(full)) + } + + /** + * Get a Long gauge. + * @param gaugeName name of the gauge + * @return gauge, if found + * @throws ClassCastException if the gauge is found but of the wrong type + */ + def getLongGauge(gaugeName: String): Option[Gauge[Long]] = { + getGauge(gaugeName).asInstanceOf[Option[Gauge[Long]]] + } + + /** + * Get a timer. + * @param timerName name of the timer + * @return the timer, if found. + */ + def getTimer(timerName: String): Option[Timer] = { + val full = fullname(timerName) + Option(metricRegistry.getTimers(new MetricByName(full)).get(full)) + } + + /** + * A filter for metrics by name; include the prefix in the name. + * @param fullname full name of metric + */ + private class MetricByName(fullname: String) extends MetricFilter { + def matches(metricName: String, metric: Metric): Boolean = metricName == fullname + } +} + + + +/** + * A Long gauge from a lambda expression; the expression is evaluated + * whenever the metrics are queried. + * @param expression expression which generates the value. + */ +private[history] class LambdaLongGauge(expression: (() => Long)) extends Gauge[Long] { + override def getValue: Long = expression() +} + +/** + * A timestamp is a gauge which is set to a point in time + * as measured in millseconds since the epoch began. + */ +private[history] class TimestampGauge(clock: Clock) extends Gauge[Long] { + var time = 0L + + /** Current value. */ + override def getValue: Long = time + + /** Set a new value. */ + def setValue(t: Long): Unit = { + time = t + } + + /** Set to the current system time. */ + def touch(): Unit = { + setValue(clock.getTimeMillis()) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 0eecf29e761b5..751ea948ce203 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -18,15 +18,12 @@ package org.apache.spark.deploy.history import java.util.NoSuchElementException -import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import scala.collection.JavaConverters._ import scala.util.control.NonFatal import scala.xml.Node -import com.codahale.metrics.{Counter, Counting, Gauge, Metric, MetricFilter, MetricRegistry, Timer} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} @@ -67,10 +64,10 @@ class HistoryServer( // application private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) - private val initialized = new AtomicBoolean(false) private[history] val metricsSystem = MetricsSystem.createMetricsSystem("history", conf, securityManager) + private[history] var metricsRegistry = metricsSystem.getMetricRegistry // and its metrics, for testing as well as monitoring @@ -121,8 +118,11 @@ class HistoryServer( } val historyMetrics = new HistoryMetrics(this, "history.server") - // provider metrics are None until the provider is started, and only after that - // point if the provider returns any. + + /** + * Provider metrics are None until the provider is started, and only after that + * point if the provider returns any. + */ var providerMetrics: Option[Source] = None initialize() @@ -134,34 +134,52 @@ class HistoryServer( * this UI with the event logs in the provided base directory. */ def initialize() { - if (!initialized.getAndSet(true)) { - attachPage(new HistoryPage(this)) + attachPage(new HistoryPage(this)) - attachHandler(ApiRootResource.getServletHandler(this)) + attachHandler(ApiRootResource.getServletHandler(this)) - attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - val contextHandler = new ServletContextHandler - contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) - contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") - attachHandler(contextHandler) - - // hook up metrics - metricsSystem.registerSource(historyMetrics) - metricsSystem.registerSource(appCache.metrics) - providerMetrics = provider.start() - providerMetrics.foreach(metricsSystem.registerSource) - metricsSystem.start() - metricsSystem.getServletHandlers.foreach(attachHandler) - } + val contextHandler = new ServletContextHandler + contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) + contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") + attachHandler(contextHandler) } - /** Bind to the HTTP server behind this web interface. */ + /** + * Startup Actions. + * 1. Call `start()` on the provider (and maybe get some metrics back). + * 2. Start the metrics. + * 3. Bind to the HTTP server behind this web interface. + */ override def bind() { + providerMetrics = provider.start() + startMetrics() super.bind() } - /** Stop the server and close the history provider. */ + /** + * Start up the metrics. + * This includes registering any metrics defined in `providerMetrics`; the provider + * needs its `start()` method to be invoked to get these metric *prior to this method + * being invoked*. + */ + private def startMetrics(): Unit = { + // hook up metrics + metricsSystem.registerSource(historyMetrics) + metricsSystem.registerSource(appCache.metrics) + providerMetrics.foreach(metricsSystem.registerSource) + metricsSystem.start() + metricsSystem.getServletHandlers.foreach(attachHandler) + } + + /** + * Stop the server. + * And: + * 1. Stop the application cache. + * 2. Stop the history provider. + * 3. Stop the metrics system. + */ override def stop() { try { super.stop() @@ -170,9 +188,7 @@ class HistoryServer( if (provider != null) { provider.stop() } - if (metricsSystem != null) { - metricsSystem.stop() - } + metricsSystem.stop() } } @@ -282,94 +298,7 @@ class HistoryServer( } } -/** - * An abstract implementation of the metrics [[Source]] trait with some common operations. - */ -private[history] abstract class HistoryMetricSource(val prefix: String) extends Source { - override val metricRegistry = new MetricRegistry() - - /** - * Register a sequence of metrics - * @param metrics sequence of metrics to register - */ - def register(metrics: Seq[(String, Metric)]): Unit = { - metrics.foreach { case (name, metric) => - metricRegistry.register(fullname(name), metric) - } - } - - /** - * Create the full name of a metric by prepending the prefix to the name - * @param name short name - * @return the full name to use in registration - */ - def fullname(name: String): String = { - MetricRegistry.name(prefix, name) - } - /** - * Dump the counters and gauges. - * @return a string for logging and diagnostics -not for parsing by machines. - */ - override def toString: String = { - val sb = new StringBuilder(s"Metrics for $sourceName:\n") - sb.append(" Counters\n") - metricRegistry.getCounters.asScala.foreach { entry => - sb.append(" ").append(entry._1).append(" = ").append(entry._2.getCount).append('\n') - } - sb.append(" Gauges\n") - metricRegistry.getGauges.asScala.foreach { entry => - sb.append(" ").append(entry._1).append(" = ").append(entry._2.getValue).append('\n') - } - sb.toString() - } - - /** - * Get a named counter. - * @param counterName name of the counter - * @return the counter, if found - */ - def getCounter(counterName: String): Option[Counter] = { - Option(metricRegistry.getCounters(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name == counterName - }).get(counterName)) - } - - /** - * Get a gauge of an unknown numeric type. - * @param gaugeName name of the gauge - * @return gauge, if found - */ - def getGauge(gaugeName: String): Option[Gauge[_]] = { - Option(metricRegistry.getGauges(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name == gaugeName - }).get(gaugeName)) - } - - /** - * Get a Long gauge. - * @param gaugeName name of the gauge - * @return gauge, if found - * @throws ClassCastException if the gauge is found but of the wrong type - */ - def getLongGauge(gaugeName: String): Option[Gauge[Long]] = { - Option(metricRegistry.getGauges(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name == gaugeName - }).get(gaugeName)).asInstanceOf[Option[Gauge[Long]]] - } - - /** - * Get a timer. - * @param timerName name of the timer - * @return the timer, if found. - */ - def getTimer(timerName: String): Option[Timer] = { - Option(metricRegistry.getTimers(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name == timerName - }).get(timerName)) - } - -} /** * History system metrics independent of providers go in here. @@ -381,44 +310,14 @@ private[history] class HistoryMetrics(val owner: HistoryServer, prefix: String) } -/** - * A timestamp is a gauge which is set to a point in time - * as measured in millseconds since the epoch began. - */ -private[history] class Timestamp extends Gauge[Long] { - var time = 0L - - /** Current value. */ - override def getValue: Long = time - - /** Set a new value. */ - def setValue(t: Long): Unit = { - time = t - } - - /** Set to the current system time. */ - def touch(): Unit = { - setValue(System.currentTimeMillis()) - } -} - -/** - * A Long gauge from a lambda expression; the expression is evaluated - * whenever the metrics are queried - * @param expression expression which generates the value. - */ -private[history] class LambdaLongGauge(expression: (() => Long)) extends Gauge[Long] { - override def getValue: Long = expression() -} - /** * The recommended way of starting and stopping a HistoryServer is through the scripts - * start-history-server.sh and stop-history-server.sh. The path to a base log directory, + * `start-history-server.sh` and `stop-history-server.sh`. The path to a base log directory, * as well as any other relevant history server configuration, should be specified via - * the $SPARK_HISTORY_OPTS environment variable. For example: + * the `SPARK_HISTORY_OPTS` environment variable. For example: * - * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" - * ./sbin/start-history-server.sh + * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" + * ./sbin/start-history-server.sh * * This launches the HistoryServer as a Spark daemon. */ diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index cda23b2001301..3f91d9edba168 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -212,7 +212,7 @@ private[spark] class MetricsSystem private ( } } - private[spark] def getMetricRegistry(): MetricRegistry = { registry } + private[spark] def getMetricRegistry(): MetricRegistry = registry } private[spark] object MetricsSystem { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryMetricSourceSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryMetricSourceSuite.scala new file mode 100644 index 0000000000000..04276f019c5d6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryMetricSourceSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import scala.collection.JavaConverters._ + +import com.codahale.metrics.{Counter, Timer} +import org.scalatest.Matchers + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.ManualClock + +class HistoryMetricSourceSuite extends SparkFunSuite with Matchers { + + test("LambdaLongGauge") { + assert(3L === new LambdaLongGauge(() => 3L).getValue) + } + + test("TimestampGauge lifecycle") { + val clock = new ManualClock(1) + val ts = new TimestampGauge(clock) + assert (0L === ts.getValue) + ts.touch() + assert (1L === ts.getValue) + clock.setTime(1000) + assert (1L === ts.getValue) + ts.touch() + assert (1000 === ts.getValue) + } + + test("HistoryMetricSource registration and lookup") { + val threeGauge = new LambdaLongGauge(() => 3L) + val clock = new ManualClock(1) + val ts = new TimestampGauge(clock) + val timer = new Timer + val counter = new Counter() + val source = new TestMetricSource + source.register(Seq( + ("three", threeGauge), + ("timestamp", ts), + ("timer", timer), + ("counter", counter))) + logInfo(source.toString) + + val registry = source.metricRegistry + val counters = registry.getCounters.asScala + counters.size should be (1) + assert(counter === counters("t.counter")) + assert(counter === source.getCounter("counter").get) + + val gauges = registry.getGauges.asScala + gauges.size should be(2) + + assert(ts === source.getLongGauge("timestamp").get) + assert(threeGauge === source.getLongGauge("three").get) + assert(timer === source.getTimer("timer").get) + + } + + test("Handle failing Gauge.getValue in toString()") { + var zero = 0L + val trouble = new LambdaLongGauge(() => 1L/zero) + intercept[ArithmeticException](trouble.getValue) + val source = new TestMetricSource + source.register(Seq(("trouble", trouble))) + val s = source.toString + logInfo(s) + s should include("ArithmeticException") + } + + private class TestMetricSource extends HistoryMetricSource("t") { + override def sourceName: String = "TestMetricSource" + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 29a4c1d18fa6e..3772b4c284e76 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -460,7 +460,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } // check that a dynamically calculated average returns 0 and not a division by zero error. - val replayTime = providerMetrics.fullname("appui.event.replay.time") + val replayTime = "appui.event.replay.time" assertGaugeEvaluates(providerMetrics, replayTime, providerMetrics.getLongGauge(replayTime).get, _ == 0) @@ -591,7 +591,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val metricsDump = s"$historyMetrics\n$providerMetrics\n$cacheMetrics" logInfo(s"Metrics:\n$metricsDump") // make some assertions about internal state of providers via the metrics - val loadcount = providerMetrics.fullname("appui.load.count") + val loadcount = "appui.load.count" assert(metricsDump.contains(loadcount), s"No $loadcount in metrics dump <$metricsDump>") assertCounterEvaluates(providerMetrics, loadcount, providerMetrics.getCounter(loadcount).get, _ > 0) From 50727e7608a9f4483c3438767968bfa626c2e459 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sat, 27 Aug 2016 12:55:12 +0100 Subject: [PATCH 18/22] [SPARK-1137] cull surplus lines --- .../scala/org/apache/spark/deploy/history/HistoryServer.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 751ea948ce203..134722b00f34d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -298,8 +298,6 @@ class HistoryServer( } } - - /** * History system metrics independent of providers go in here. * @param owner owning instance From 012cf92250c051461e344d5814e74dc8f66433de Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 30 Aug 2016 12:21:06 +0100 Subject: [PATCH 19/22] [SPARK-11373] don't register appui.load.timer as a metric --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index ee2c7133dc105..52ddb13ccbce4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -898,9 +898,7 @@ private[history] class FsHistoryProviderMetrics(owner: FsHistoryProvider, prefix ("appui.event.count", appUIEventCount), ("appui.event.replay.time", appUIEventReplayTime), ("update.timer", updateTimer), - ("history.merge.timer", historyMergeTimer), - ("appui.load.timer", appUiLoadTimer))) - + ("history.merge.timer", historyMergeTimer))) } private[history] object FsHistoryProvider { From 1e59b687a27e8640558b5eef58670a6d824a52d3 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 10 Oct 2016 19:25:51 +0100 Subject: [PATCH 20/22] [SPARK-11373] sync with master; tests all happy --- .../spark/deploy/history/ApplicationCache.scala | 2 +- .../spark/deploy/history/FsHistoryProvider.scala | 15 +-------------- .../spark/deploy/history/HistoryServer.scala | 1 - 3 files changed, 2 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 6a44ecd029d13..9b2affab5bc17 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -24,7 +24,7 @@ import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import com.codahale.metrics.{Counter, Counting, Metric, MetricRegistry, Timer} +import com.codahale.metrics.{Counter, Counting, Metric, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} import org.eclipse.jetty.servlet.FilterHolder diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 52ddb13ccbce4..084a2e6bf823a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -245,16 +245,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - /** - * Bind to the History Server: start update and cleaner threads; perform any metric registration - * - * @param binding binding information - */ - override def start(binding: ApplicationHistoryBinding): Unit = { - super.start(binding) - historyMetrics.registerMetrics(binding.metrics, binding.health) - } - override def getListing(): Iterator[FsApplicationHistoryInfo] = applications.values.iterator override def getApplicationInfo(appId: String): Option[FsApplicationHistoryInfo] = { @@ -288,10 +278,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (appListener.appId.isDefined) { ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) // make sure to set admin acls before view acls so they are properly picked up - val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") - ui.getSecurityManager.setAdminAcls(adminAcls) - ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) - val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + + val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("")val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + appListener.adminAclsGroups.getOrElse("") ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 134722b00f34d..5442594593714 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -305,7 +305,6 @@ class HistoryServer( private[history] class HistoryMetrics(val owner: HistoryServer, prefix: String) extends HistoryMetricSource(prefix) { override val sourceName = "history" - } /** From ec1f2d7f8743ce6de3e83f2f9a82f1c940c8be52 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 1 Mar 2017 13:53:48 +0000 Subject: [PATCH 21/22] [SPARK-11373] sync this PR up with the master branch --- .../deploy/history/FsHistoryProvider.scala | 18 ++++++++---------- .../deploy/history/HistoryMetricSource.scala | 3 +-- .../spark/deploy/history/HistoryServer.scala | 8 ++++---- .../history/FsHistoryProviderSuite.scala | 15 ++++++++++++--- .../deploy/history/HistoryServerSuite.scala | 12 ++++++++---- 5 files changed, 33 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 084a2e6bf823a..843e63b95d795 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -28,7 +28,7 @@ import scala.xml.Node import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.io.ByteStreams import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException @@ -278,8 +278,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (appListener.appId.isDefined) { ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE) // make sure to set admin acls before view acls so they are properly picked up - val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("")val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - appListener.adminAclsGroups.getOrElse("") + val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") + ui.getSecurityManager.setAdminAcls(adminAcls) + ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) + val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + appListener.adminAclsGroups.getOrElse("") ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) Some(LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize))) @@ -345,7 +347,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val logInfos: Seq[FileStatus] = statusList .filter { entry => try { - val prevFileSize = fileToAppInfo.get(entry.getPath()) .map {_.fileSize}.getOrElse(0L) + val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L) !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would log an error which can be scary to @@ -411,11 +413,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) metrics.updateLastSucceeded.setValue(newLastScanTime) } catch { case e: Exception => logError( - "Exception in checking for event log updates", - e) + "Exception in checking for event log updates", e) metrics.updateFailureCount.inc() - } finally { - updateContext.stop() } } } @@ -478,7 +477,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - /** * Replay the log files in the list and merge the list of old applications with new ones */ @@ -608,7 +606,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) appsToRetain += (app.id -> app) } else if (toRetain.nonEmpty) { appsToRetain += (app.id -> - new FsApplicationHistoryInfo(app.id, app.name, toRetain)) + new FsApplicationHistoryInfo(app.id, app.name, toRetain.toList)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala index cada33d917d0c..361089bef4f99 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryMetricSource.scala @@ -29,6 +29,7 @@ import org.apache.spark.util.Clock * retrieving entries; the `toString()` operation dumps all counters and gauges. */ private[history] abstract class HistoryMetricSource(val prefix: String) extends Source { + override val metricRegistry = new MetricRegistry() /** @@ -134,8 +135,6 @@ private[history] abstract class HistoryMetricSource(val prefix: String) extends } } - - /** * A Long gauge from a lambda expression; the expression is evaluated * whenever the metrics are queried. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 5442594593714..e13e86c47ddec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -309,12 +309,12 @@ private[history] class HistoryMetrics(val owner: HistoryServer, prefix: String) /** * The recommended way of starting and stopping a HistoryServer is through the scripts - * `start-history-server.sh` and `stop-history-server.sh`. The path to a base log directory, + * start-history-server.sh and stop-history-server.sh. The path to a base log directory, * as well as any other relevant history server configuration, should be specified via - * the `SPARK_HISTORY_OPTS` environment variable. For example: + * the $SPARK_HISTORY_OPTS environment variable. For example: * - * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" - * ./sbin/start-history-server.sh + * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" + * ./sbin/start-history-server.sh * * This launches the HistoryServer as a Spark daemon. */ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 00e80f6a04ed8..92f0b35d184c2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -70,7 +70,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } /** - * Create a configure a new history provider + * Create and configure a new history provider * @return a filesystem history provider ready for use */ private def createHistoryProvider(): FsHistoryProvider = { @@ -79,10 +79,19 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc provider } + /** + * Create and configure a new history provider + * @return a filesystem history provider ready for use + */ + private def createHistoryProvider(clock: Clock): FsHistoryProvider = { + val provider = new FsHistoryProvider(createTestConf(), clock) + provider.start() + provider + } + test("Parse application logs") { val clock = new ManualClock(12345678) - val provider = new FsHistoryProvider(createTestConf(), clock) - val provider = createHistoryProvider() + val provider = createHistoryProvider(clock) // Write a new-style application log. val newAppComplete = newLogFile("new1", None, inProgress = false) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 3772b4c284e76..f702e9d76b554 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -81,9 +81,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val securityManager = HistoryServer.createSecurityManager(conf) server = new HistoryServer(conf, provider, securityManager, 18080) + server.initialize() server.bind() port = server.boundPort - provider.checkForLogs() } def stop(): Unit = { @@ -417,6 +417,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // stop the server with the old config, and start the new one server.stop() server = new HistoryServer(myConf, provider, securityManager, 18080) + server.initialize() server.bind() val port = server.boundPort val cacheMetrics = server.cacheMetrics @@ -537,7 +538,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers getNumJobs("") should be (1) getNumJobs("/jobs") should be (1) getNumJobsRestful() should be (1) - assertCounterEvaluates(cacheMetrics, "lookup count", cacheMetrics.lookupCount, _ > 1) + assertCounterEvaluates(cacheMetrics, "lookup count", + cacheMetrics.lookupCount, _ > 1) // dump state before the next bit of test, which is where update // checking really gets stressed @@ -592,14 +594,16 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers logInfo(s"Metrics:\n$metricsDump") // make some assertions about internal state of providers via the metrics val loadcount = "appui.load.count" - assert(metricsDump.contains(loadcount), s"No $loadcount in metrics dump <$metricsDump>") + assert(metricsDump.contains(loadcount), + s"No $loadcount in metrics dump <$metricsDump>") assertCounterEvaluates(providerMetrics, loadcount, providerMetrics.getCounter(loadcount).get, _ > 0) assertGaugeEvaluates(providerMetrics, replayTime, providerMetrics.getLongGauge(replayTime).get, _ > 0) val evictionCount = cacheMetrics.fullname("eviction.count") - assert(metricsDump.contains(evictionCount), s"No $evictionCount in metrics dump <$metricsDump>") + assert(metricsDump.contains(evictionCount), + s"No $evictionCount in metrics dump <$metricsDump>") // no need to retain the test dir now the tests complete logDir.deleteOnExit(); From 8903dcfe2b927c8fc3fed9df3e9939670a016944 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 11 Apr 2017 13:55:28 +0100 Subject: [PATCH 22/22] HADOOP-11374 fix line length errors Change-Id: I7c3fc0865abce380fcbe08b8984cfd00b3ce0faa --- .../spark/deploy/history/FsHistoryProvider.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 843e63b95d795..d890c10beedcc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -272,7 +272,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) - val (appListener, count) = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) + val (appListener, count) = replay(fileStatus, + isApplicationCompleted(fileStatus), replayBus) metrics.appUIEventCount.inc(count) if (appListener.appId.isDefined) { @@ -280,8 +281,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // make sure to set admin acls before view acls so they are properly picked up val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("") ui.getSecurityManager.setAdminAcls(adminAcls) - ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) - val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + appListener.adminAclsGroups.getOrElse("") + ui.getSecurityManager.setViewAcls(attempt.sparkUser, + appListener.viewAcls.getOrElse("")) + val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," + + appListener.adminAclsGroups.getOrElse("") ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups) ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) Some(LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize))) @@ -495,7 +498,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // won't change whenever HistoryServer restarts and reloads the file. val lastUpdated = if (appCompleted) fileStatus.getModificationTime else clock.getTimeMillis() - val (appListener, count) = replay(fileStatus, appCompleted, new ReplayListenerBus(), eventsFilter) + val (appListener, count) = replay(fileStatus, appCompleted, + new ReplayListenerBus(), eventsFilter) metrics.historyEventCount.inc(count) // Without an app ID, new logs will render incorrectly in the listing page, so do not list or