blob: af69d911e307612aae2ac99af0ea9ecb7d1657d6 [file] [log] [blame]
<!DOCTYPE html>
<!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]-->
<!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]-->
<!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>Monitoring and Instrumentation - Spark 3.5.0 Documentation</title>
<meta name="description" content="Monitoring, metrics, and instrumentation guide for Spark 3.5.0">
<link rel="stylesheet" href="css/bootstrap.min.css">
<link rel="preconnect" href="https://fonts.googleapis.com">
<link rel="preconnect" href="https://fonts.gstatic.com" crossorigin>
<link href="https://fonts.googleapis.com/css2?family=DM+Sans:ital,wght@0,400;0,500;0,700;1,400;1,500;1,700&Courier+Prime:wght@400;700&display=swap" rel="stylesheet">
<link href="css/custom.css" rel="stylesheet">
<script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="css/pygments-default.css">
<link rel="stylesheet" href="css/docsearch.min.css" />
<link rel="stylesheet" href="css/docsearch.css">
<!-- Matomo -->
<script type="text/javascript">
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
_paq.push(["disableCookies"]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="https://analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '40']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
<!-- End Matomo Code -->
</head>
<body class="global">
<!--[if lt IE 7]>
<p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p>
<![endif]-->
<!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
<nav class="navbar navbar-expand-lg navbar-dark p-0 px-4 fixed-top" style="background: #1d6890;" id="topbar">
<div class="navbar-brand"><a href="index.html">
<img src="img/spark-logo-rev.svg" width="141" height="72"/></a><span class="version">3.5.0</span>
</div>
<button class="navbar-toggler" type="button" data-toggle="collapse"
data-target="#navbarCollapse" aria-controls="navbarCollapse"
aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div class="collapse navbar-collapse" id="navbarCollapse">
<ul class="navbar-nav me-auto">
<li class="nav-item"><a href="index.html" class="nav-link">Overview</a></li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarQuickStart" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Programming Guides</a>
<div class="dropdown-menu" aria-labelledby="navbarQuickStart">
<a class="dropdown-item" href="quick-start.html">Quick Start</a>
<a class="dropdown-item" href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a>
<a class="dropdown-item" href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a>
<a class="dropdown-item" href="structured-streaming-programming-guide.html">Structured Streaming</a>
<a class="dropdown-item" href="streaming-programming-guide.html">Spark Streaming (DStreams)</a>
<a class="dropdown-item" href="ml-guide.html">MLlib (Machine Learning)</a>
<a class="dropdown-item" href="graphx-programming-guide.html">GraphX (Graph Processing)</a>
<a class="dropdown-item" href="sparkr.html">SparkR (R on Spark)</a>
<a class="dropdown-item" href="api/python/getting_started/index.html">PySpark (Python on Spark)</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarAPIDocs" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">API Docs</a>
<div class="dropdown-menu" aria-labelledby="navbarAPIDocs">
<a class="dropdown-item" href="api/scala/org/apache/spark/index.html">Scala</a>
<a class="dropdown-item" href="api/java/index.html">Java</a>
<a class="dropdown-item" href="api/python/index.html">Python</a>
<a class="dropdown-item" href="api/R/index.html">R</a>
<a class="dropdown-item" href="api/sql/index.html">SQL, Built-in Functions</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarDeploying" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Deploying</a>
<div class="dropdown-menu" aria-labelledby="navbarDeploying">
<a class="dropdown-item" href="cluster-overview.html">Overview</a>
<a class="dropdown-item" href="submitting-applications.html">Submitting Applications</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="spark-standalone.html">Spark Standalone</a>
<a class="dropdown-item" href="running-on-mesos.html">Mesos</a>
<a class="dropdown-item" href="running-on-yarn.html">YARN</a>
<a class="dropdown-item" href="running-on-kubernetes.html">Kubernetes</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarMore" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">More</a>
<div class="dropdown-menu" aria-labelledby="navbarMore">
<a class="dropdown-item" href="configuration.html">Configuration</a>
<a class="dropdown-item" href="monitoring.html">Monitoring</a>
<a class="dropdown-item" href="tuning.html">Tuning Guide</a>
<a class="dropdown-item" href="job-scheduling.html">Job Scheduling</a>
<a class="dropdown-item" href="security.html">Security</a>
<a class="dropdown-item" href="hardware-provisioning.html">Hardware Provisioning</a>
<a class="dropdown-item" href="migration-guide.html">Migration Guide</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="building-spark.html">Building Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/contributing.html">Contributing to Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a>
</div>
</li>
<li class="nav-item">
<input type="text" id="docsearch-input" placeholder="Search the docs…">
</li>
</ul>
<!--<span class="navbar-text navbar-right"><span class="version-text">v3.5.0</span></span>-->
</div>
</nav>
<div class="container">
<div class="content mr-3" id="content">
<h1 class="title">Monitoring and Instrumentation</h1>
<ul id="markdown-toc">
<li><a href="#web-interfaces" id="markdown-toc-web-interfaces">Web Interfaces</a> <ul>
<li><a href="#viewing-after-the-fact" id="markdown-toc-viewing-after-the-fact">Viewing After the Fact</a> <ul>
<li><a href="#environment-variables" id="markdown-toc-environment-variables">Environment Variables</a></li>
<li><a href="#applying-compaction-on-rolling-event-log-files" id="markdown-toc-applying-compaction-on-rolling-event-log-files">Applying compaction on rolling event log files</a></li>
<li><a href="#spark-history-server-configuration-options" id="markdown-toc-spark-history-server-configuration-options">Spark History Server Configuration Options</a></li>
</ul>
</li>
<li><a href="#rest-api" id="markdown-toc-rest-api">REST API</a> <ul>
<li><a href="#executor-task-metrics" id="markdown-toc-executor-task-metrics">Executor Task Metrics</a></li>
<li><a href="#executor-metrics" id="markdown-toc-executor-metrics">Executor Metrics</a></li>
<li><a href="#api-versioning-policy" id="markdown-toc-api-versioning-policy">API Versioning Policy</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#metrics" id="markdown-toc-metrics">Metrics</a> <ul>
<li><a href="#list-of-available-metrics-providers" id="markdown-toc-list-of-available-metrics-providers">List of available metrics providers</a> <ul>
<li><a href="#component-instance--driver" id="markdown-toc-component-instance--driver">Component instance = Driver</a></li>
<li><a href="#component-instance--executor" id="markdown-toc-component-instance--executor">Component instance = Executor</a></li>
<li><a href="#source--jvm-source" id="markdown-toc-source--jvm-source">Source = JVM Source</a></li>
<li><a href="#component-instance--applicationmaster" id="markdown-toc-component-instance--applicationmaster">Component instance = applicationMaster</a></li>
<li><a href="#component-instance--mesos_cluster" id="markdown-toc-component-instance--mesos_cluster">Component instance = mesos_cluster</a></li>
<li><a href="#component-instance--master" id="markdown-toc-component-instance--master">Component instance = master</a></li>
<li><a href="#component-instance--applicationsource" id="markdown-toc-component-instance--applicationsource">Component instance = ApplicationSource</a></li>
<li><a href="#component-instance--worker" id="markdown-toc-component-instance--worker">Component instance = worker</a></li>
<li><a href="#component-instance--shuffleservice" id="markdown-toc-component-instance--shuffleservice">Component instance = shuffleService</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#advanced-instrumentation" id="markdown-toc-advanced-instrumentation">Advanced Instrumentation</a></li>
</ul>
<p>There are several ways to monitor Spark applications: web UIs, metrics, and external instrumentation.</p>
<h1 id="web-interfaces">Web Interfaces</h1>
<p>Every SparkContext launches a <a href="web-ui.html">Web UI</a>, by default on port 4040, that
displays useful information about the application. This includes:</p>
<ul>
<li>A list of scheduler stages and tasks</li>
<li>A summary of RDD sizes and memory usage</li>
<li>Environmental information.</li>
<li>Information about the running executors</li>
</ul>
<p>You can access this interface by simply opening <code class="language-plaintext highlighter-rouge">http://&lt;driver-node&gt;:4040</code> in a web browser.
If multiple SparkContexts are running on the same host, they will bind to successive ports
beginning with 4040 (4041, 4042, etc).</p>
<p>Note that this information is only available for the duration of the application by default.
To view the web UI after the fact, set <code class="language-plaintext highlighter-rouge">spark.eventLog.enabled</code> to true before starting the
application. This configures Spark to log Spark events that encode the information displayed
in the UI to persisted storage.</p>
<h2 id="viewing-after-the-fact">Viewing After the Fact</h2>
<p>It is still possible to construct the UI of an application through Spark&#8217;s history server,
provided that the application&#8217;s event logs exist.
You can start the history server by executing:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./sbin/start-history-server.sh
</code></pre></div></div>
<p>This creates a web interface at <code class="language-plaintext highlighter-rouge">http://&lt;server-url&gt;:18080</code> by default, listing incomplete
and completed applications and attempts.</p>
<p>When using the file-system provider class (see <code class="language-plaintext highlighter-rouge">spark.history.provider</code> below), the base logging
directory must be supplied in the <code class="language-plaintext highlighter-rouge">spark.history.fs.logDirectory</code> configuration option,
and should contain sub-directories that each represents an application&#8217;s event logs.</p>
<p>The spark jobs themselves must be configured to log events, and to log them to the same shared,
writable directory. For example, if the server was configured with a log directory of
<code class="language-plaintext highlighter-rouge">hdfs://namenode/shared/spark-logs</code>, then the client-side options would be:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>spark.eventLog.enabled true
spark.eventLog.dir hdfs://namenode/shared/spark-logs
</code></pre></div></div>
<p>The history server can be configured as follows:</p>
<h3 id="environment-variables">Environment Variables</h3>
<table class="table table-striped">
<thead><tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr></thead>
<tr>
<td><code>SPARK_DAEMON_MEMORY</code></td>
<td>Memory to allocate to the history server (default: 1g).</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
<td>JVM options for the history server (default: none).</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_CLASSPATH</code></td>
<td>Classpath for the history server (default: none).</td>
</tr>
<tr>
<td><code>SPARK_PUBLIC_DNS</code></td>
<td>
The public address for the history server. If this is not set, links to application history
may use the internal address of the server, resulting in broken links (default: none).
</td>
</tr>
<tr>
<td><code>SPARK_HISTORY_OPTS</code></td>
<td>
<code>spark.history.*</code> configuration options for the history server (default: none).
</td>
</tr>
</table>
<h3 id="applying-compaction-on-rolling-event-log-files">Applying compaction on rolling event log files</h3>
<p>A long-running application (e.g. streaming) can bring a huge single event log file which may cost a lot to maintain and
also requires a bunch of resource to replay per each update in Spark History Server.</p>
<p>Enabling <code>spark.eventLog.rolling.enabled</code> and <code>spark.eventLog.rolling.maxFileSize</code> would
let you have rolling event log files instead of single huge event log file which may help some scenarios on its own,
but it still doesn&#8217;t help you reducing the overall size of logs.</p>
<p>Spark History Server can apply compaction on the rolling event log files to reduce the overall size of
logs, via setting the configuration <code>spark.history.fs.eventLog.rolling.maxFilesToRetain</code> on the
Spark History Server.</p>
<p>Details will be described below, but please note in prior that compaction is LOSSY operation.
Compaction will discard some events which will be no longer seen on UI - you may want to check which events will be discarded
before enabling the option.</p>
<p>When the compaction happens, the History Server lists all the available event log files for the application, and considers
the event log files having less index than the file with smallest index which will be retained as target of compaction.
For example, if the application A has 5 event log files and <code>spark.history.fs.eventLog.rolling.maxFilesToRetain</code> is set to 2, then first 3 log files will be selected to be compacted.</p>
<p>Once it selects the target, it analyzes them to figure out which events can be excluded, and rewrites them
into one compact file with discarding events which are decided to exclude.</p>
<p>The compaction tries to exclude the events which point to the outdated data. As of now, below describes the candidates of events to be excluded:</p>
<ul>
<li>Events for the job which is finished, and related stage/tasks events</li>
<li>Events for the executor which is terminated</li>
<li>Events for the SQL execution which is finished, and related job/stage/tasks events</li>
</ul>
<p>Once rewriting is done, original log files will be deleted, via best-effort manner. The History Server may not be able to delete
the original log files, but it will not affect the operation of the History Server.</p>
<p>Please note that Spark History Server may not compact the old event log files if figures out not a lot of space
would be reduced during compaction. For streaming query we normally expect compaction
will run as each micro-batch will trigger one or more jobs which will be finished shortly, but compaction won&#8217;t run
in many cases for batch query.</p>
<p>Please also note that this is a new feature introduced in Spark 3.0, and may not be completely stable. Under some circumstances,
the compaction may exclude more events than you expect, leading some UI issues on History Server for the application.
Use it with caution.</p>
<h3 id="spark-history-server-configuration-options">Spark History Server Configuration Options</h3>
<p>Security options for the Spark History Server are covered more detail in the
<a href="security.html#web-ui">Security</a> page.</p>
<table class="table table-striped">
<thead>
<tr>
<th>Property Name</th>
<th>Default</th>
<th>Meaning</th>
<th>Since Version</th>
</tr>
</thead>
<tr>
<td>spark.history.provider</td>
<td><code>org.apache.spark.deploy.history.FsHistoryProvider</code></td>
<td>Name of the class implementing the application history backend. Currently there is only
one implementation, provided by Spark, which looks for application logs stored in the
file system.</td>
<td>1.1.0</td>
</tr>
<tr>
<td>spark.history.fs.logDirectory</td>
<td>file:/tmp/spark-events</td>
<td>
For the filesystem history provider, the URL to the directory containing application event
logs to load. This can be a local <code>file://</code> path,
an HDFS path <code>hdfs://namenode/shared/spark-logs</code>
or that of an alternative filesystem supported by the Hadoop APIs.
</td>
<td>1.1.0</td>
</tr>
<tr>
<td>spark.history.fs.update.interval</td>
<td>10s</td>
<td>
The period at which the filesystem history provider checks for new or
updated logs in the log directory. A shorter interval detects new applications faster,
at the expense of more server load re-reading updated applications.
As soon as an update has completed, listings of the completed and incomplete applications
will reflect the changes.
</td>
<td>1.4.0</td>
</tr>
<tr>
<td>spark.history.retainedApplications</td>
<td>50</td>
<td>
The number of applications to retain UI data for in the cache. If this cap is exceeded, then
the oldest applications will be removed from the cache. If an application is not in the cache,
it will have to be loaded from disk if it is accessed from the UI.
</td>
<td>1.0.0</td>
</tr>
<tr>
<td>spark.history.ui.maxApplications</td>
<td>Int.MaxValue</td>
<td>
The number of applications to display on the history summary page. Application UIs are still
available by accessing their URLs directly even if they are not displayed on the history summary page.
</td>
<td>2.0.1</td>
</tr>
<tr>
<td>spark.history.ui.port</td>
<td>18080</td>
<td>
The port to which the web interface of the history server binds.
</td>
<td>1.0.0</td>
</tr>
<tr>
<td>spark.history.kerberos.enabled</td>
<td>false</td>
<td>
Indicates whether the history server should use kerberos to login. This is required
if the history server is accessing HDFS files on a secure Hadoop cluster.
</td>
<td>1.0.1</td>
</tr>
<tr>
<td>spark.history.kerberos.principal</td>
<td>(none)</td>
<td>
When <code>spark.history.kerberos.enabled=true</code>, specifies kerberos principal name for the History Server.
</td>
<td>1.0.1</td>
</tr>
<tr>
<td>spark.history.kerberos.keytab</td>
<td>(none)</td>
<td>
When <code>spark.history.kerberos.enabled=true</code>, specifies location of the kerberos keytab file for the History Server.
</td>
<td>1.0.1</td>
</tr>
<tr>
<td>spark.history.fs.cleaner.enabled</td>
<td>false</td>
<td>
Specifies whether the History Server should periodically clean up event logs from storage.
</td>
<td>1.4.0</td>
</tr>
<tr>
<td>spark.history.fs.cleaner.interval</td>
<td>1d</td>
<td>
When <code>spark.history.fs.cleaner.enabled=true</code>, specifies how often the filesystem job history cleaner checks for files to delete.
Files are deleted if at least one of two conditions holds.
First, they're deleted if they're older than <code>spark.history.fs.cleaner.maxAge</code>.
They are also deleted if the number of files is more than
<code>spark.history.fs.cleaner.maxNum</code>, Spark tries to clean up the completed attempts
from the applications based on the order of their oldest attempt time.
</td>
<td>1.4.0</td>
</tr>
<tr>
<td>spark.history.fs.cleaner.maxAge</td>
<td>7d</td>
<td>
When <code>spark.history.fs.cleaner.enabled=true</code>, job history files older than this will be deleted when the filesystem history cleaner runs.
</td>
<td>1.4.0</td>
</tr>
<tr>
<td>spark.history.fs.cleaner.maxNum</td>
<td>Int.MaxValue</td>
<td>
When <code>spark.history.fs.cleaner.enabled=true</code>, specifies the maximum number of files in the event log directory.
Spark tries to clean up the completed attempt logs to maintain the log directory under this limit.
This should be smaller than the underlying file system limit like
`dfs.namenode.fs-limits.max-directory-items` in HDFS.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.fs.endEventReparseChunkSize</td>
<td>1m</td>
<td>
How many bytes to parse at the end of log files looking for the end event.
This is used to speed up generation of application listings by skipping unnecessary
parts of event log files. It can be disabled by setting this config to 0.
</td>
<td>2.4.0</td>
</tr>
<tr>
<td>spark.history.fs.inProgressOptimization.enabled</td>
<td>true</td>
<td>
Enable optimized handling of in-progress logs. This option may leave finished
applications that fail to rename their event logs listed as in-progress.
</td>
<td>2.4.0</td>
</tr>
<tr>
<td>spark.history.fs.driverlog.cleaner.enabled</td>
<td><code>spark.history.fs.cleaner.enabled</code></td>
<td>
Specifies whether the History Server should periodically clean up driver logs from storage.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.fs.driverlog.cleaner.interval</td>
<td><code>spark.history.fs.cleaner.interval</code></td>
<td>
When <code>spark.history.fs.driverlog.cleaner.enabled=true</code>, specifies how often the filesystem driver log cleaner checks for files to delete.
Files are only deleted if they are older than <code>spark.history.fs.driverlog.cleaner.maxAge</code>
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.fs.driverlog.cleaner.maxAge</td>
<td><code>spark.history.fs.cleaner.maxAge</code></td>
<td>
When <code>spark.history.fs.driverlog.cleaner.enabled=true</code>, driver log files older than this will be deleted when the driver log cleaner runs.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.fs.numReplayThreads</td>
<td>25% of available cores</td>
<td>
Number of threads that will be used by history server to process event logs.
</td>
<td>2.0.0</td>
</tr>
<tr>
<td>spark.history.store.maxDiskUsage</td>
<td>10g</td>
<td>
Maximum disk usage for the local directory where the cache application history information
are stored.
</td>
<td>2.3.0</td>
</tr>
<tr>
<td>spark.history.store.path</td>
<td>(none)</td>
<td>
Local directory where to cache application history data. If set, the history
server will store application data on disk instead of keeping it in memory. The data
written to disk will be re-used in the event of a history server restart.
</td>
<td>2.3.0</td>
</tr>
<tr>
<td>spark.history.store.serializer</td>
<td>JSON</td>
<td>
Serializer for writing/reading in-memory UI objects to/from disk-based KV Store; JSON or PROTOBUF.
JSON serializer is the only choice before Spark 3.4.0, thus it is the default value.
PROTOBUF serializer is fast and compact, compared to the JSON serializer.
</td>
<td>3.4.0</td>
</tr>
<tr>
<td>spark.history.custom.executor.log.url</td>
<td>(none)</td>
<td>
Specifies custom spark executor log URL for supporting external log service instead of using cluster
managers' application log URLs in the history server. Spark will support some path variables via patterns
which can vary on cluster manager. Please check the documentation for your cluster manager to
see which patterns are supported, if any. This configuration has no effect on a live application, it only
affects the history server.
<p />
For now, only YARN mode supports this configuration
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.custom.executor.log.url.applyIncompleteApplication</td>
<td>true</td>
<td>
Specifies whether to apply custom spark executor log URL to incomplete applications as well.
If executor logs for running applications should be provided as origin log URLs, set this to `false`.
Please note that incomplete applications may include applications which didn't shutdown gracefully.
Even this is set to `true`, this configuration has no effect on a live application, it only affects the history server.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.fs.eventLog.rolling.maxFilesToRetain</td>
<td>Int.MaxValue</td>
<td>
The maximum number of event log files which will be retained as non-compacted. By default,
all event log files will be retained. The lowest value is 1 for technical reason.<br />
Please read the section of "Applying compaction of old event log files" for more details.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td>spark.history.store.hybridStore.enabled</td>
<td>false</td>
<td>
Whether to use HybridStore as the store when parsing event logs. HybridStore will first write data
to an in-memory store and having a background thread that dumps data to a disk store after the writing
to in-memory store is completed.
</td>
<td>3.1.0</td>
</tr>
<tr>
<td>spark.history.store.hybridStore.maxMemoryUsage</td>
<td>2g</td>
<td>
Maximum memory space that can be used to create HybridStore. The HybridStore co-uses the heap memory,
so the heap memory should be increased through the memory option for SHS if the HybridStore is enabled.
</td>
<td>3.1.0</td>
</tr>
<tr>
<td>spark.history.store.hybridStore.diskBackend</td>
<td>LEVELDB</td>
<td>
Specifies a disk-based store used in hybrid store; LEVELDB or ROCKSDB.
</td>
<td>3.3.0</td>
</tr>
<tr>
<td>spark.history.fs.update.batchSize</td>
<td>Int.MaxValue</td>
<td>
Specifies the batch size for updating new eventlog files.
This controls each scan process to be completed within a reasonable time, and such
prevent the initial scan from running too long and blocking new eventlog files to
be scanned in time in large environments.
</td>
<td>3.4.0</td>
</tr>
</table>
<p>Note that in all of these UIs, the tables are sortable by clicking their headers,
making it easy to identify slow tasks, data skew, etc.</p>
<p>Note</p>
<ol>
<li>
<p>The history server displays both completed and incomplete Spark jobs. If an application makes
multiple attempts after failures, the failed attempts will be displayed, as well as any ongoing
incomplete attempt or the final successful attempt.</p>
</li>
<li>
<p>Incomplete applications are only updated intermittently. The time between updates is defined
by the interval between checks for changed files (<code class="language-plaintext highlighter-rouge">spark.history.fs.update.interval</code>).
On larger clusters, the update interval may be set to large values.
The way to view a running application is actually to view its own web UI.</p>
</li>
<li>
<p>Applications which exited without registering themselves as completed will be listed
as incomplete —even though they are no longer running. This can happen if an application
crashes.</p>
</li>
<li>
<p>One way to signal the completion of a Spark job is to stop the Spark Context
explicitly (<code class="language-plaintext highlighter-rouge">sc.stop()</code>), or in Python using the <code class="language-plaintext highlighter-rouge">with SparkContext() as sc:</code> construct
to handle the Spark Context setup and tear down.</p>
</li>
</ol>
<h2 id="rest-api">REST API</h2>
<p>In addition to viewing the metrics in the UI, they are also available as JSON. This gives developers
an easy way to create new visualizations and monitoring tools for Spark. The JSON is available for
both running applications, and in the history server. The endpoints are mounted at <code class="language-plaintext highlighter-rouge">/api/v1</code>. For example,
for the history server, they would typically be accessible at <code class="language-plaintext highlighter-rouge">http://&lt;server-url&gt;:18080/api/v1</code>, and
for a running application, at <code class="language-plaintext highlighter-rouge">http://localhost:4040/api/v1</code>.</p>
<p>In the API, an application is referenced by its application ID, <code class="language-plaintext highlighter-rouge">[app-id]</code>.
When running on YARN, each application may have multiple attempts, but there are attempt IDs
only for applications in cluster mode, not applications in client mode. Applications in YARN cluster mode
can be identified by their <code class="language-plaintext highlighter-rouge">[attempt-id]</code>. In the API listed below, when running in YARN cluster mode,
<code class="language-plaintext highlighter-rouge">[app-id]</code> will actually be <code class="language-plaintext highlighter-rouge">[base-app-id]/[attempt-id]</code>, where <code class="language-plaintext highlighter-rouge">[base-app-id]</code> is the YARN application ID.</p>
<table class="table table-striped">
<thead><tr><th>Endpoint</th><th>Meaning</th></tr></thead>
<tr>
<td><code>/applications</code></td>
<td>A list of all applications.
<br />
<code>?status=[completed|running]</code> list only applications in the chosen state.
<br />
<code>?minDate=[date]</code> earliest start date/time to list.
<br />
<code>?maxDate=[date]</code> latest start date/time to list.
<br />
<code>?minEndDate=[date]</code> earliest end date/time to list.
<br />
<code>?maxEndDate=[date]</code> latest end date/time to list.
<br />
<code>?limit=[limit]</code> limits the number of applications listed.
<br />Examples:
<br /><code>?minDate=2015-02-10</code>
<br /><code>?minDate=2015-02-03T16:42:40.000GMT</code>
<br /><code>?maxDate=2015-02-11T20:41:30.000GMT</code>
<br /><code>?minEndDate=2015-02-12</code>
<br /><code>?minEndDate=2015-02-12T09:15:10.000GMT</code>
<br /><code>?maxEndDate=2015-02-14T16:30:45.000GMT</code>
<br /><code>?limit=10</code></td>
</tr>
<tr>
<td><code>/applications/[app-id]/jobs</code></td>
<td>
A list of all jobs for a given application.
<br /><code>?status=[running|succeeded|failed|unknown]</code> list only jobs in the specific state.
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/jobs/[job-id]</code></td>
<td>Details for the given job.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/stages</code></td>
<td>
A list of all stages for a given application.
<br /><code>?status=[active|complete|pending|failed]</code> list only stages in the given state.
<br /><code>?details=true</code> lists all stages with the task data.
<br /><code>?taskStatus=[RUNNING|SUCCESS|FAILED|KILLED|PENDING]</code> lists only those tasks with the specified task status. Query parameter taskStatus takes effect only when <code>details=true</code>. This also supports multiple <code>taskStatus</code> such as <code>?details=true&amp;taskStatus=SUCCESS&amp;taskStatus=FAILED</code> which will return all tasks matching any of specified task status.
<br /><code>?withSummaries=true</code> lists stages with task metrics distribution and executor metrics distribution.
<br /><code>?quantiles=0.0,0.25,0.5,0.75,1.0</code> summarize the metrics with the given quantiles. Query parameter quantiles takes effect only when <code>withSummaries=true</code>. Default value is <code>0.0,0.25,0.5,0.75,1.0</code>.
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/stages/[stage-id]</code></td>
<td>
A list of all attempts for the given stage.
<br /><code>?details=true</code> lists all attempts with the task data for the given stage.
<br /><code>?taskStatus=[RUNNING|SUCCESS|FAILED|KILLED|PENDING]</code> lists only those tasks with the specified task status. Query parameter taskStatus takes effect only when <code>details=true</code>. This also supports multiple <code>taskStatus</code> such as <code>?details=true&amp;taskStatus=SUCCESS&amp;taskStatus=FAILED</code> which will return all tasks matching any of specified task status.
<br /><code>?withSummaries=true</code> lists task metrics distribution and executor metrics distribution of each attempt.
<br /><code>?quantiles=0.0,0.25,0.5,0.75,1.0</code> summarize the metrics with the given quantiles. Query parameter quantiles takes effect only when <code>withSummaries=true</code>. Default value is <code>0.0,0.25,0.5,0.75,1.0</code>.
<br />Example:
<br /><code>?details=true</code>
<br /><code>?details=true&amp;taskStatus=RUNNING</code>
<br /><code>?withSummaries=true</code>
<br /><code>?details=true&amp;withSummaries=true&amp;quantiles=0.01,0.5,0.99</code>
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]</code></td>
<td>
Details for the given stage attempt.
<br /><code>?details=true</code> lists all task data for the given stage attempt.
<br /><code>?taskStatus=[RUNNING|SUCCESS|FAILED|KILLED|PENDING]</code> lists only those tasks with the specified task status. Query parameter taskStatus takes effect only when <code>details=true</code>. This also supports multiple <code>taskStatus</code> such as <code>?details=true&amp;taskStatus=SUCCESS&amp;taskStatus=FAILED</code> which will return all tasks matching any of specified task status.
<br /><code>?withSummaries=true</code> lists task metrics distribution and executor metrics distribution for the given stage attempt.
<br /><code>?quantiles=0.0,0.25,0.5,0.75,1.0</code> summarize the metrics with the given quantiles. Query parameter quantiles takes effect only when <code>withSummaries=true</code>. Default value is <code>0.0,0.25,0.5,0.75,1.0</code>.
<br />Example:
<br /><code>?details=true</code>
<br /><code>?details=true&amp;taskStatus=RUNNING</code>
<br /><code>?withSummaries=true</code>
<br /><code>?details=true&amp;withSummaries=true&amp;quantiles=0.01,0.5,0.99</code>
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummary</code></td>
<td>
Summary metrics of all tasks in the given stage attempt.
<br /><code>?quantiles</code> summarize the metrics with the given quantiles.
<br />Example: <code>?quantiles=0.01,0.5,0.99</code>
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskList</code></td>
<td>
A list of all tasks for the given stage attempt.
<br /><code>?offset=[offset]&amp;length=[len]</code> list tasks in the given range.
<br /><code>?sortBy=[runtime|-runtime]</code> sort the tasks.
<br /><code>?status=[running|success|killed|failed|unknown]</code> list only tasks in the state.
<br />Example: <code>?offset=10&amp;length=50&amp;sortBy=runtime&amp;status=running</code>
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/executors</code></td>
<td>A list of all active executors for the given application.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/executors/[executor-id]/threads</code></td>
<td>
Stack traces of all the threads running within the given active executor.
Not available via the history server.
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/allexecutors</code></td>
<td>A list of all(active and dead) executors for the given application.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/storage/rdd</code></td>
<td>A list of stored RDDs for the given application.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/storage/rdd/[rdd-id]</code></td>
<td>Details for the storage status of a given RDD.</td>
</tr>
<tr>
<td><code>/applications/[base-app-id]/logs</code></td>
<td>Download the event logs for all attempts of the given application as files within
a zip file.
</td>
</tr>
<tr>
<td><code>/applications/[base-app-id]/[attempt-id]/logs</code></td>
<td>Download the event logs for a specific application attempt as a zip file.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/statistics</code></td>
<td>Statistics for the streaming context.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/receivers</code></td>
<td>A list of all streaming receivers.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/receivers/[stream-id]</code></td>
<td>Details of the given receiver.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/batches</code></td>
<td>A list of all retained batches.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/batches/[batch-id]</code></td>
<td>Details of the given batch.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/batches/[batch-id]/operations</code></td>
<td>A list of all output operations of the given batch.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/streaming/batches/[batch-id]/operations/[outputOp-id]</code></td>
<td>Details of the given operation and given batch.</td>
</tr>
<tr>
<td><code>/applications/[app-id]/sql</code></td>
<td>A list of all queries for a given application.
<br />
<code>?details=[true (default) | false]</code> lists/hides details of Spark plan nodes.
<br />
<code>?planDescription=[true (default) | false]</code> enables/disables Physical <code>planDescription</code> on demand when Physical Plan size is high.
<br />
<code>?offset=[offset]&amp;length=[len]</code> lists queries in the given range.
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/sql/[execution-id]</code></td>
<td>Details for the given query.
<br />
<code>?details=[true (default) | false]</code> lists/hides metric details in addition to given query details.
<br />
<code>?planDescription=[true (default) | false]</code> enables/disables Physical <code>planDescription</code> on demand for the given query when Physical Plan size is high.
</td>
</tr>
<tr>
<td><code>/applications/[app-id]/environment</code></td>
<td>Environment details of the given application.</td>
</tr>
<tr>
<td><code>/version</code></td>
<td>Get the current spark version.</td>
</tr>
</table>
<p>The number of jobs and stages which can be retrieved is constrained by the same retention
mechanism of the standalone Spark UI; <code class="language-plaintext highlighter-rouge">"spark.ui.retainedJobs"</code> defines the threshold
value triggering garbage collection on jobs, and <code class="language-plaintext highlighter-rouge">spark.ui.retainedStages</code> that for stages.
Note that the garbage collection takes place on playback: it is possible to retrieve
more entries by increasing these values and restarting the history server.</p>
<h3 id="executor-task-metrics">Executor Task Metrics</h3>
<p>The REST API exposes the values of the Task Metrics collected by Spark executors with the granularity
of task execution. The metrics can be used for performance troubleshooting and workload characterization.
A list of the available metrics, with a short description:</p>
<table class="table table-striped">
<thead>
<tr>
<th>Spark Executor Task Metric name</th>
<th>Short description</th>
</tr>
</thead>
<tr>
<td>executorRunTime</td>
<td>Elapsed time the executor spent running this task. This includes time fetching shuffle data.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>executorCpuTime</td>
<td>CPU time the executor spent running this task. This includes time fetching shuffle data.
The value is expressed in nanoseconds.</td>
</tr>
<tr>
<td>executorDeserializeTime</td>
<td>Elapsed time spent to deserialize this task. The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>executorDeserializeCpuTime</td>
<td>CPU time taken on the executor to deserialize this task. The value is expressed
in nanoseconds.</td>
</tr>
<tr>
<td>resultSize</td>
<td>The number of bytes this task transmitted back to the driver as the TaskResult.</td>
</tr>
<tr>
<td>jvmGCTime</td>
<td>Elapsed time the JVM spent in garbage collection while executing this task.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>resultSerializationTime</td>
<td>Elapsed time spent serializing the task result. The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>memoryBytesSpilled</td>
<td>The number of in-memory bytes spilled by this task.</td>
</tr>
<tr>
<td>diskBytesSpilled</td>
<td>The number of on-disk bytes spilled by this task.</td>
</tr>
<tr>
<td>peakExecutionMemory</td>
<td>Peak memory used by internal data structures created during shuffles, aggregations and
joins. The value of this accumulator should be approximately the sum of the peak sizes
across all such data structures created in this task. For SQL jobs, this only tracks all
unsafe operators and ExternalSort.</td>
</tr>
<tr>
<td>inputMetrics.*</td>
<td>Metrics related to reading data from <code>org.apache.spark.rdd.HadoopRDD</code>
or from persisted data.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.bytesRead</td>
<td>Total number of bytes read.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.recordsRead</td>
<td>Total number of records read.</td>
</tr>
<tr>
<td>outputMetrics.*</td>
<td>Metrics related to writing data externally (e.g. to a distributed filesystem),
defined only in tasks with output.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.bytesWritten</td>
<td>Total number of bytes written</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.recordsWritten</td>
<td>Total number of records written</td>
</tr>
<tr>
<td>shuffleReadMetrics.*</td>
<td>Metrics related to shuffle read operations.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.recordsRead</td>
<td>Number of records read in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.remoteBlocksFetched</td>
<td>Number of remote blocks fetched in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.localBlocksFetched</td>
<td>Number of local (as opposed to read from a remote executor) blocks fetched
in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.totalBlocksFetched</td>
<td>Number of blocks fetched in shuffle operations (both local and remote)</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.remoteBytesRead</td>
<td>Number of remote bytes read in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.localBytesRead</td>
<td>Number of bytes read in shuffle operations from local disk (as opposed to
read from a remote executor)</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.totalBytesRead</td>
<td>Number of bytes read in shuffle operations (both local and remote)</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.remoteBytesReadToDisk</td>
<td>Number of remote bytes read to disk in shuffle operations.
Large blocks are fetched to disk in shuffle read operations, as opposed to
being read into memory, which is the default behavior.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.fetchWaitTime</td>
<td>Time the task spent waiting for remote shuffle blocks.
This only includes the time blocking on shuffle input data.
For instance if block B is being fetched while the task is still not finished
processing block A, it is not considered to be blocking on block B.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>shuffleWriteMetrics.*</td>
<td>Metrics related to operations writing shuffle data.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.bytesWritten</td>
<td>Number of bytes written in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.recordsWritten</td>
<td>Number of records written in shuffle operations</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.writeTime</td>
<td>Time spent blocking on writes to disk or buffer cache. The value is expressed
in nanoseconds.</td>
</tr>
</table>
<h3 id="executor-metrics">Executor Metrics</h3>
<p>Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information.
Executor metric values and their measured memory peak values per executor are exposed via the REST API in JSON format and in Prometheus format.
The JSON end point is exposed at: <code class="language-plaintext highlighter-rouge">/applications/[app-id]/executors</code>, and the Prometheus endpoint at: <code class="language-plaintext highlighter-rouge">/metrics/executors/prometheus</code>.
The Prometheus endpoint is conditional to a configuration parameter: <code class="language-plaintext highlighter-rouge">spark.ui.prometheus.enabled=true</code> (the default is <code class="language-plaintext highlighter-rouge">false</code>).
In addition, aggregated per-stage peak values of the executor memory metrics are written to the event log if
<code class="language-plaintext highlighter-rouge">spark.eventLog.logStageExecutorMetrics</code> is true.
Executor memory metrics are also exposed via the Spark metrics system based on the <a href="https://metrics.dropwizard.io/4.2.0">Dropwizard metrics library</a>.
A list of the available metrics, with a short description:</p>
<table class="table table-striped">
<thead>
<tr><th>Executor Level Metric name</th>
<th>Short description</th>
</tr>
</thead>
<tr>
<td>rddBlocks</td>
<td>RDD blocks in the block manager of this executor.</td>
</tr>
<tr>
<td>memoryUsed</td>
<td>Storage memory used by this executor.</td>
</tr>
<tr>
<td>diskUsed</td>
<td>Disk space used for RDD storage by this executor.</td>
</tr>
<tr>
<td>totalCores</td>
<td>Number of cores available in this executor.</td>
</tr>
<tr>
<td>maxTasks</td>
<td>Maximum number of tasks that can run concurrently in this executor.</td>
</tr>
<tr>
<td>activeTasks</td>
<td>Number of tasks currently executing.</td>
</tr>
<tr>
<td>failedTasks</td>
<td>Number of tasks that have failed in this executor.</td>
</tr>
<tr>
<td>completedTasks</td>
<td>Number of tasks that have completed in this executor.</td>
</tr>
<tr>
<td>totalTasks</td>
<td>Total number of tasks (running, failed and completed) in this executor.</td>
</tr>
<tr>
<td>totalDuration</td>
<td>Elapsed time the JVM spent executing tasks in this executor.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>totalGCTime</td>
<td>Elapsed time the JVM spent in garbage collection summed in this executor.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>totalInputBytes</td>
<td>Total input bytes summed in this executor.</td>
</tr>
<tr>
<td>totalShuffleRead</td>
<td>Total shuffle read bytes summed in this executor.</td>
</tr>
<tr>
<td>totalShuffleWrite</td>
<td>Total shuffle write bytes summed in this executor.</td>
</tr>
<tr>
<td>maxMemory</td>
<td>Total amount of memory available for storage, in bytes.</td>
</tr>
<tr>
<td>memoryMetrics.*</td>
<td>Current value of memory metrics:</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.usedOnHeapStorageMemory</td>
<td>Used on heap memory currently for storage, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.usedOffHeapStorageMemory</td>
<td>Used off heap memory currently for storage, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.totalOnHeapStorageMemory</td>
<td>Total available on heap memory for storage, in bytes. This amount can vary over time, on the MemoryManager implementation.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.totalOffHeapStorageMemory</td>
<td>Total available off heap memory for storage, in bytes. This amount can vary over time, depending on the MemoryManager implementation.</td>
</tr>
<tr>
<td>peakMemoryMetrics.*</td>
<td>Peak value of memory (and GC) metrics:</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.JVMHeapMemory</td>
<td>Peak memory usage of the heap that is used for object allocation.
The heap consists of one or more memory pools. The used and committed size of the returned memory usage is the sum of those values of all heap memory pools whereas the init and max size of the returned memory usage represents the setting of the heap memory which may not be the sum of those of all heap memory pools.
The amount of used memory in the returned memory usage is the amount of memory occupied by both live objects and garbage objects that have not been collected, if any.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.JVMOffHeapMemory</td>
<td>Peak memory usage of non-heap memory that is used by the Java virtual machine. The non-heap memory consists of one or more memory pools. The used and committed size of the returned memory usage is the sum of those values of all non-heap memory pools whereas the init and max size of the returned memory usage represents the setting of the non-heap memory which may not be the sum of those of all non-heap memory pools.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OnHeapExecutionMemory</td>
<td>Peak on heap execution memory in use, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OffHeapExecutionMemory</td>
<td>Peak off heap execution memory in use, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OnHeapStorageMemory</td>
<td>Peak on heap storage memory in use, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OffHeapStorageMemory</td>
<td>Peak off heap storage memory in use, in bytes.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OnHeapUnifiedMemory</td>
<td>Peak on heap memory (execution and storage).</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.OffHeapUnifiedMemory</td>
<td>Peak off heap memory (execution and storage).</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.DirectPoolMemory</td>
<td>Peak memory that the JVM is using for direct buffer pool (<code>java.lang.management.BufferPoolMXBean</code>)</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.MappedPoolMemory</td>
<td>Peak memory that the JVM is using for mapped buffer pool (<code>java.lang.management.BufferPoolMXBean</code>)</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreeJVMVMemory</td>
<td>Virtual memory size in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreeJVMRSSMemory</td>
<td>Resident Set Size: number of pages the process has
in real memory. This is just the pages which count
toward text, data, or stack space. This does not
include pages which have not been demand-loaded in,
or which are swapped out. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreePythonVMemory</td>
<td>Virtual memory size for Python in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreePythonRSSMemory</td>
<td>Resident Set Size for Python. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreeOtherVMemory</td>
<td>Virtual memory size for other kind of process in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.ProcessTreeOtherRSSMemory</td>
<td>Resident Set Size for other kind of process. Enabled if spark.executor.processTreeMetrics.enabled is true.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.MinorGCCount</td>
<td>Total minor GC count. For example, the garbage collector is one of Copy, PS Scavenge, ParNew, G1 Young Generation and so on.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.MinorGCTime</td>
<td>Elapsed total minor GC time.
The value is expressed in milliseconds.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.MajorGCCount</td>
<td>Total major GC count. For example, the garbage collector is one of MarkSweepCompact, PS MarkSweep, ConcurrentMarkSweep, G1 Old Generation and so on.</td>
</tr>
<tr>
<td>&nbsp;&nbsp;&nbsp;&nbsp;.MajorGCTime</td>
<td>Elapsed total major GC time.
The value is expressed in milliseconds.</td>
</tr>
</table>
<p>The computation of RSS and Vmem are based on <a href="http://man7.org/linux/man-pages/man5/proc.5.html">proc(5)</a></p>
<h3 id="api-versioning-policy">API Versioning Policy</h3>
<p>These endpoints have been strongly versioned to make it easier to develop applications on top.
In particular, Spark guarantees:</p>
<ul>
<li>Endpoints will never be removed from one version</li>
<li>Individual fields will never be removed for any given endpoint</li>
<li>New endpoints may be added</li>
<li>New fields may be added to existing endpoints</li>
<li>New versions of the api may be added in the future as a separate endpoint (e.g., <code class="language-plaintext highlighter-rouge">api/v2</code>). New versions are <em>not</em> required to be backwards compatible.</li>
<li>Api versions may be dropped, but only after at least one minor release of co-existing with a new api version.</li>
</ul>
<p>Note that even when examining the UI of running applications, the <code class="language-plaintext highlighter-rouge">applications/[app-id]</code> portion is
still required, though there is only one application available. E.g. to see the list of jobs for the
running app, you would go to <code class="language-plaintext highlighter-rouge">http://localhost:4040/api/v1/applications/[app-id]/jobs</code>. This is to
keep the paths consistent in both modes.</p>
<h1 id="metrics">Metrics</h1>
<p>Spark has a configurable metrics system based on the
<a href="https://metrics.dropwizard.io/4.2.0">Dropwizard Metrics Library</a>.
This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV
files. The metrics are generated by sources embedded in the Spark code base. They
provide instrumentation for specific activities and Spark components.
The metrics system is configured via a configuration file that Spark expects to be present
at <code class="language-plaintext highlighter-rouge">$SPARK_HOME/conf/metrics.properties</code>. A custom file location can be specified via the
<code class="language-plaintext highlighter-rouge">spark.metrics.conf</code> <a href="configuration.html#spark-properties">configuration property</a>.
Instead of using the configuration file, a set of configuration parameters with prefix
<code class="language-plaintext highlighter-rouge">spark.metrics.conf.</code> can be used.
By default, the root namespace used for driver or executor metrics is
the value of <code class="language-plaintext highlighter-rouge">spark.app.id</code>. However, often times, users want to be able to track the metrics
across apps for driver and executors, which is hard to do with application ID
(i.e. <code class="language-plaintext highlighter-rouge">spark.app.id</code>) since it changes with every invocation of the app. For such use cases,
a custom namespace can be specified for metrics reporting using <code class="language-plaintext highlighter-rouge">spark.metrics.namespace</code>
configuration property.
If, say, users wanted to set the metrics namespace to the name of the application, they
can set the <code class="language-plaintext highlighter-rouge">spark.metrics.namespace</code> property to a value like <code class="language-plaintext highlighter-rouge">${spark.app.name}</code>. This value is
then expanded appropriately by Spark and is used as the root namespace of the metrics system.
Non-driver and executor metrics are never prefixed with <code class="language-plaintext highlighter-rouge">spark.app.id</code>, nor does the
<code class="language-plaintext highlighter-rouge">spark.metrics.namespace</code> property have any such affect on such metrics.</p>
<p>Spark&#8217;s metrics are decoupled into different
<em>instances</em> corresponding to Spark components. Within each instance, you can configure a
set of sinks to which metrics are reported. The following instances are currently supported:</p>
<ul>
<li><code class="language-plaintext highlighter-rouge">master</code>: The Spark standalone master process.</li>
<li><code class="language-plaintext highlighter-rouge">applications</code>: A component within the master which reports on various applications.</li>
<li><code class="language-plaintext highlighter-rouge">worker</code>: A Spark standalone worker process.</li>
<li><code class="language-plaintext highlighter-rouge">executor</code>: A Spark executor.</li>
<li><code class="language-plaintext highlighter-rouge">driver</code>: The Spark driver process (the process in which your SparkContext is created).</li>
<li><code class="language-plaintext highlighter-rouge">shuffleService</code>: The Spark shuffle service.</li>
<li><code class="language-plaintext highlighter-rouge">applicationMaster</code>: The Spark ApplicationMaster when running on YARN.</li>
<li><code class="language-plaintext highlighter-rouge">mesos_cluster</code>: The Spark cluster scheduler when running on Mesos.</li>
</ul>
<p>Each instance can report to zero or more <em>sinks</em>. Sinks are contained in the
<code class="language-plaintext highlighter-rouge">org.apache.spark.metrics.sink</code> package:</p>
<ul>
<li><code class="language-plaintext highlighter-rouge">ConsoleSink</code>: Logs metrics information to the console.</li>
<li><code class="language-plaintext highlighter-rouge">CSVSink</code>: Exports metrics data to CSV files at regular intervals.</li>
<li><code class="language-plaintext highlighter-rouge">JmxSink</code>: Registers metrics for viewing in a JMX console.</li>
<li><code class="language-plaintext highlighter-rouge">MetricsServlet</code>: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.</li>
<li><code class="language-plaintext highlighter-rouge">PrometheusServlet</code>: (Experimental) Adds a servlet within the existing Spark UI to serve metrics data in Prometheus format.</li>
<li><code class="language-plaintext highlighter-rouge">GraphiteSink</code>: Sends metrics to a Graphite node.</li>
<li><code class="language-plaintext highlighter-rouge">Slf4jSink</code>: Sends metrics to slf4j as log entries.</li>
<li><code class="language-plaintext highlighter-rouge">StatsdSink</code>: Sends metrics to a StatsD node.</li>
</ul>
<p>Spark also supports a Ganglia sink which is not included in the default build due to
licensing restrictions:</p>
<ul>
<li><code class="language-plaintext highlighter-rouge">GangliaSink</code>: Sends metrics to a Ganglia node or multicast group.</li>
</ul>
<p>To install the <code class="language-plaintext highlighter-rouge">GangliaSink</code> you&#8217;ll need to perform a custom build of Spark. <em><strong>Note that
by embedding this library you will include <a href="http://www.gnu.org/copyleft/lesser.html">LGPL</a>-licensed
code in your Spark package</strong></em>. For sbt users, set the
<code class="language-plaintext highlighter-rouge">SPARK_GANGLIA_LGPL</code> environment variable before building. For Maven users, enable
the <code class="language-plaintext highlighter-rouge">-Pspark-ganglia-lgpl</code> profile. In addition to modifying the cluster&#8217;s Spark build
user applications will need to link to the <code class="language-plaintext highlighter-rouge">spark-ganglia-lgpl</code> artifact.</p>
<p>The syntax of the metrics configuration file and the parameters available for each sink are defined
in an example configuration file,
<code class="language-plaintext highlighter-rouge">$SPARK_HOME/conf/metrics.properties.template</code>.</p>
<p>When using Spark configuration parameters instead of the metrics configuration file, the relevant
parameter names are composed by the prefix <code class="language-plaintext highlighter-rouge">spark.metrics.conf.</code> followed by the configuration
details, i.e. the parameters take the following form:
<code class="language-plaintext highlighter-rouge">spark.metrics.conf.[instance|*].sink.[sink_name].[parameter_name]</code>.
This example shows a list of Spark configuration parameters for a Graphite sink:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>"spark.metrics.conf.*.sink.graphite.class"="org.apache.spark.metrics.sink.GraphiteSink"
"spark.metrics.conf.*.sink.graphite.host"="graphiteEndPoint_hostName&gt;"
"spark.metrics.conf.*.sink.graphite.port"=&lt;graphite_listening_port&gt;
"spark.metrics.conf.*.sink.graphite.period"=10
"spark.metrics.conf.*.sink.graphite.unit"=seconds
"spark.metrics.conf.*.sink.graphite.prefix"="optional_prefix"
"spark.metrics.conf.*.sink.graphite.regex"="optional_regex_to_send_matching_metrics"
</code></pre></div></div>
<p>Default values of the Spark metrics configuration are as follows:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>"*.sink.servlet.class" = "org.apache.spark.metrics.sink.MetricsServlet"
"*.sink.servlet.path" = "/metrics/json"
"master.sink.servlet.path" = "/metrics/master/json"
"applications.sink.servlet.path" = "/metrics/applications/json"
</code></pre></div></div>
<p>Additional sources can be configured using the metrics configuration file or the configuration
parameter <code class="language-plaintext highlighter-rouge">spark.metrics.conf.[component_name].source.jvm.class=[source_name]</code>. At present the
JVM source is the only available optional source. For example the following configuration parameter
activates the JVM source:
<code class="language-plaintext highlighter-rouge">"spark.metrics.conf.*.source.jvm.class"="org.apache.spark.metrics.source.JvmSource"</code></p>
<h2 id="list-of-available-metrics-providers">List of available metrics providers</h2>
<p>Metrics used by Spark are of multiple types: gauge, counter, histogram, meter and timer,
see <a href="https://metrics.dropwizard.io/4.2.0/getting-started.html">Dropwizard library documentation for details</a>.
The following list of components and metrics reports the name and some details about the available metrics,
grouped per component instance and source namespace.
The most common time of metrics used in Spark instrumentation are gauges and counters.
Counters can be recognized as they have the <code class="language-plaintext highlighter-rouge">.count</code> suffix. Timers, meters and histograms are annotated
in the list, the rest of the list elements are metrics of type gauge.
The large majority of metrics are active as soon as their parent component instance is configured,
some metrics require also to be enabled via an additional configuration parameter, the details are
reported in the list.</p>
<h3 id="component-instance--driver">Component instance = Driver</h3>
<p>This is the component with the largest amount of instrumented metrics</p>
<ul>
<li>namespace=BlockManager
<ul>
<li>disk.diskSpaceUsed_MB</li>
<li>memory.maxMem_MB</li>
<li>memory.maxOffHeapMem_MB</li>
<li>memory.maxOnHeapMem_MB</li>
<li>memory.memUsed_MB</li>
<li>memory.offHeapMemUsed_MB</li>
<li>memory.onHeapMemUsed_MB</li>
<li>memory.remainingMem_MB</li>
<li>memory.remainingOffHeapMem_MB</li>
<li>memory.remainingOnHeapMem_MB</li>
</ul>
</li>
<li>namespace=HiveExternalCatalog
<ul>
<li><strong>note:</strong> these metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.staticSources.enabled</code> (default is true)</li>
<li>fileCacheHits.count</li>
<li>filesDiscovered.count</li>
<li>hiveClientCalls.count</li>
<li>parallelListingJobCount.count</li>
<li>partitionsFetched.count</li>
</ul>
</li>
<li>namespace=CodeGenerator
<ul>
<li><strong>note:</strong> these metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.staticSources.enabled</code> (default is true)</li>
<li>compilationTime (histogram)</li>
<li>generatedClassSize (histogram)</li>
<li>generatedMethodSize (histogram)</li>
<li>sourceCodeSize (histogram)</li>
</ul>
</li>
<li>namespace=DAGScheduler
<ul>
<li>job.activeJobs</li>
<li>job.allJobs</li>
<li>messageProcessingTime (timer)</li>
<li>stage.failedStages</li>
<li>stage.runningStages</li>
<li>stage.waitingStages</li>
</ul>
</li>
<li>namespace=LiveListenerBus
<ul>
<li>listenerProcessingTime.org.apache.spark.HeartbeatReceiver (timer)</li>
<li>listenerProcessingTime.org.apache.spark.scheduler.EventLoggingListener (timer)</li>
<li>listenerProcessingTime.org.apache.spark.status.AppStatusListener (timer)</li>
<li>numEventsPosted.count</li>
<li>queue.appStatus.listenerProcessingTime (timer)</li>
<li>queue.appStatus.numDroppedEvents.count</li>
<li>queue.appStatus.size</li>
<li>queue.eventLog.listenerProcessingTime (timer)</li>
<li>queue.eventLog.numDroppedEvents.count</li>
<li>queue.eventLog.size</li>
<li>queue.executorManagement.listenerProcessingTime (timer)</li>
</ul>
</li>
<li>namespace=appStatus (all metrics of type=counter)
<ul>
<li><strong>note:</strong> Introduced in Spark 3.0. Conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.appStatusSource.enabled</code> (default is false)</li>
<li>stages.failedStages.count</li>
<li>stages.skippedStages.count</li>
<li>stages.completedStages.count</li>
<li>tasks.blackListedExecutors.count // deprecated use excludedExecutors instead</li>
<li>tasks.excludedExecutors.count</li>
<li>tasks.completedTasks.count</li>
<li>tasks.failedTasks.count</li>
<li>tasks.killedTasks.count</li>
<li>tasks.skippedTasks.count</li>
<li>tasks.unblackListedExecutors.count // deprecated use unexcludedExecutors instead</li>
<li>tasks.unexcludedExecutors.count</li>
<li>jobs.succeededJobs</li>
<li>jobs.failedJobs</li>
<li>jobDuration</li>
</ul>
</li>
<li>namespace=AccumulatorSource
<ul>
<li><strong>note:</strong> User-configurable sources to attach accumulators to metric system</li>
<li>DoubleAccumulatorSource</li>
<li>LongAccumulatorSource</li>
</ul>
</li>
<li>namespace=spark.streaming
<ul>
<li><strong>note:</strong> This applies to Spark Structured Streaming only. Conditional to a configuration
parameter: <code class="language-plaintext highlighter-rouge">spark.sql.streaming.metricsEnabled=true</code> (default is false)</li>
<li>eventTime-watermark</li>
<li>inputRate-total</li>
<li>latency</li>
<li>processingRate-total</li>
<li>states-rowsTotal</li>
<li>states-usedBytes</li>
</ul>
</li>
<li>namespace=JVMCPU
<ul>
<li>jvmCpuTime</li>
</ul>
</li>
<li>namespace=executor
<ul>
<li><strong>note:</strong> These metrics are available in the driver in local mode only.</li>
<li>A full list of available metrics in this
namespace can be found in the corresponding entry for the Executor component instance.</li>
</ul>
</li>
<li>namespace=ExecutorMetrics
<ul>
<li><strong>note:</strong> these metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.executorMetricsSource.enabled</code> (default is true)</li>
<li>This source contains memory-related metrics. A full list of available metrics in this
namespace can be found in the corresponding entry for the Executor component instance.</li>
</ul>
</li>
<li>namespace=ExecutorAllocationManager
<ul>
<li><strong>note:</strong> these metrics are only emitted when using dynamic allocation. Conditional to a configuration
parameter <code class="language-plaintext highlighter-rouge">spark.dynamicAllocation.enabled</code> (default is false)</li>
<li>executors.numberExecutorsToAdd</li>
<li>executors.numberExecutorsPendingToRemove</li>
<li>executors.numberAllExecutors</li>
<li>executors.numberTargetExecutors</li>
<li>executors.numberMaxNeededExecutors</li>
<li>executors.numberDecommissioningExecutors</li>
<li>executors.numberExecutorsGracefullyDecommissioned.count</li>
<li>executors.numberExecutorsDecommissionUnfinished.count</li>
<li>executors.numberExecutorsExitedUnexpectedly.count</li>
<li>executors.numberExecutorsKilledByDriver.count</li>
</ul>
</li>
<li>namespace=plugin.&lt;Plugin Class Name&gt;
<ul>
<li>Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and
configured using the Spark plugin API. See &#8220;Advanced Instrumentation&#8221; below for how to load
custom plugins into Spark.</li>
</ul>
</li>
</ul>
<h3 id="component-instance--executor">Component instance = Executor</h3>
<p>These metrics are exposed by Spark executors.</p>
<ul>
<li>namespace=executor (metrics are of type counter or gauge)
<ul>
<li><strong>notes:</strong>
<ul>
<li><code class="language-plaintext highlighter-rouge">spark.executor.metrics.fileSystemSchemes</code> (default: <code class="language-plaintext highlighter-rouge">file,hdfs</code>) determines the exposed file system metrics.</li>
</ul>
</li>
<li>bytesRead.count</li>
<li>bytesWritten.count</li>
<li>cpuTime.count</li>
<li>deserializeCpuTime.count</li>
<li>deserializeTime.count</li>
<li>diskBytesSpilled.count</li>
<li>filesystem.file.largeRead_ops</li>
<li>filesystem.file.read_bytes</li>
<li>filesystem.file.read_ops</li>
<li>filesystem.file.write_bytes</li>
<li>filesystem.file.write_ops</li>
<li>filesystem.hdfs.largeRead_ops</li>
<li>filesystem.hdfs.read_bytes</li>
<li>filesystem.hdfs.read_ops</li>
<li>filesystem.hdfs.write_bytes</li>
<li>filesystem.hdfs.write_ops</li>
<li>jvmGCTime.count</li>
<li>memoryBytesSpilled.count</li>
<li>recordsRead.count</li>
<li>recordsWritten.count</li>
<li>resultSerializationTime.count</li>
<li>resultSize.count</li>
<li>runTime.count</li>
<li>shuffleBytesWritten.count</li>
<li>shuffleFetchWaitTime.count</li>
<li>shuffleLocalBlocksFetched.count</li>
<li>shuffleLocalBytesRead.count</li>
<li>shuffleRecordsRead.count</li>
<li>shuffleRecordsWritten.count</li>
<li>shuffleRemoteBlocksFetched.count</li>
<li>shuffleRemoteBytesRead.count</li>
<li>shuffleRemoteBytesReadToDisk.count</li>
<li>shuffleTotalBytesRead.count</li>
<li>shuffleWriteTime.count</li>
<li>succeededTasks.count</li>
<li>threadpool.activeTasks</li>
<li>threadpool.completeTasks</li>
<li>threadpool.currentPool_size</li>
<li>threadpool.maxPool_size</li>
<li>threadpool.startedTasks</li>
</ul>
</li>
<li>namespace=ExecutorMetrics
<ul>
<li><strong>notes:</strong>
<ul>
<li>These metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.executorMetricsSource.enabled</code> (default value is true)</li>
<li>ExecutorMetrics are updated as part of heartbeat processes scheduled
for the executors and for the driver at regular intervals: <code class="language-plaintext highlighter-rouge">spark.executor.heartbeatInterval</code> (default value is 10 seconds)</li>
<li>An optional faster polling mechanism is available for executor memory metrics,
it can be activated by setting a polling interval (in milliseconds) using the configuration parameter <code class="language-plaintext highlighter-rouge">spark.executor.metrics.pollingInterval</code></li>
</ul>
</li>
<li>JVMHeapMemory</li>
<li>JVMOffHeapMemory</li>
<li>OnHeapExecutionMemory</li>
<li>OnHeapStorageMemory</li>
<li>OnHeapUnifiedMemory</li>
<li>OffHeapExecutionMemory</li>
<li>OffHeapStorageMemory</li>
<li>OffHeapUnifiedMemory</li>
<li>DirectPoolMemory</li>
<li>MappedPoolMemory</li>
<li>MinorGCCount</li>
<li>MinorGCTime</li>
<li>MajorGCCount</li>
<li>MajorGCTime</li>
<li>&#8220;ProcessTree*&#8221; metric counters:
<ul>
<li>ProcessTreeJVMVMemory</li>
<li>ProcessTreeJVMRSSMemory</li>
<li>ProcessTreePythonVMemory</li>
<li>ProcessTreePythonRSSMemory</li>
<li>ProcessTreeOtherVMemory</li>
<li>ProcessTreeOtherRSSMemory</li>
<li><strong>note:</strong> &#8220;ProcessTree<em>&#8221; metrics are collected only under certain conditions.
The conditions are the logical AND of the following: <code class="language-plaintext highlighter-rouge">/proc</code> filesystem exists,
<code class="language-plaintext highlighter-rouge">spark.executor.processTreeMetrics.enabled=true</code>.
&#8220;ProcessTree</em>&#8221; metrics report 0 when those conditions are not met.</li>
</ul>
</li>
</ul>
</li>
<li>namespace=JVMCPU
<ul>
<li>jvmCpuTime</li>
</ul>
</li>
<li>namespace=NettyBlockTransfer
<ul>
<li>shuffle-client.usedDirectMemory</li>
<li>shuffle-client.usedHeapMemory</li>
<li>shuffle-server.usedDirectMemory</li>
<li>shuffle-server.usedHeapMemory</li>
</ul>
</li>
<li>namespace=HiveExternalCatalog
<ul>
<li><strong>note:</strong> these metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.staticSources.enabled</code> (default is true)</li>
<li>fileCacheHits.count</li>
<li>filesDiscovered.count</li>
<li>hiveClientCalls.count</li>
<li>parallelListingJobCount.count</li>
<li>partitionsFetched.count</li>
</ul>
</li>
<li>namespace=CodeGenerator
<ul>
<li><strong>note:</strong> these metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.staticSources.enabled</code> (default is true)</li>
<li>compilationTime (histogram)</li>
<li>generatedClassSize (histogram)</li>
<li>generatedMethodSize (histogram)</li>
<li>sourceCodeSize (histogram)</li>
</ul>
</li>
<li>namespace=plugin.&lt;Plugin Class Name&gt;
<ul>
<li>Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and
configured using the Spark plugin API. See &#8220;Advanced Instrumentation&#8221; below for how to load
custom plugins into Spark.</li>
</ul>
</li>
</ul>
<h3 id="source--jvm-source">Source = JVM Source</h3>
<p>Notes:</p>
<ul>
<li>Activate this source by setting the relevant <code class="language-plaintext highlighter-rouge">metrics.properties</code> file entry or the
configuration parameter:<code class="language-plaintext highlighter-rouge">spark.metrics.conf.*.source.jvm.class=org.apache.spark.metrics.source.JvmSource</code></li>
<li>These metrics are conditional to a configuration parameter:
<code class="language-plaintext highlighter-rouge">spark.metrics.staticSources.enabled</code> (default is true)</li>
<li>This source is available for driver and executor instances and is also available for other instances.</li>
<li>This source provides information on JVM metrics using the
<a href="https://metrics.dropwizard.io/4.2.0/manual/jvm.html">Dropwizard/Codahale Metric Sets for JVM instrumentation</a>
and in particular the metric sets BufferPoolMetricSet, GarbageCollectorMetricSet and MemoryUsageGaugeSet.</li>
</ul>
<h3 id="component-instance--applicationmaster">Component instance = applicationMaster</h3>
<p>Note: applies when running on YARN</p>
<ul>
<li>numContainersPendingAllocate</li>
<li>numExecutorsFailed</li>
<li>numExecutorsRunning</li>
<li>numLocalityAwareTasks</li>
<li>numReleasedContainers</li>
</ul>
<h3 id="component-instance--mesos_cluster">Component instance = mesos_cluster</h3>
<p>Note: applies when running on mesos</p>
<ul>
<li>waitingDrivers</li>
<li>launchedDrivers</li>
<li>retryDrivers</li>
</ul>
<h3 id="component-instance--master">Component instance = master</h3>
<p>Note: applies when running in Spark standalone as master</p>
<ul>
<li>workers</li>
<li>aliveWorkers</li>
<li>apps</li>
<li>waitingApps</li>
</ul>
<h3 id="component-instance--applicationsource">Component instance = ApplicationSource</h3>
<p>Note: applies when running in Spark standalone as master</p>
<ul>
<li>status</li>
<li>runtime_ms</li>
<li>cores</li>
</ul>
<h3 id="component-instance--worker">Component instance = worker</h3>
<p>Note: applies when running in Spark standalone as worker</p>
<ul>
<li>executors</li>
<li>coresUsed</li>
<li>memUsed_MB</li>
<li>coresFree</li>
<li>memFree_MB</li>
</ul>
<h3 id="component-instance--shuffleservice">Component instance = shuffleService</h3>
<p>Note: applies to the shuffle service</p>
<ul>
<li>blockTransferRate (meter) - rate of blocks being transferred</li>
<li>blockTransferMessageRate (meter) - rate of block transfer messages,
i.e. if batch fetches are enabled, this represents number of batches rather than number of blocks</li>
<li>blockTransferRateBytes (meter)</li>
<li>blockTransferAvgTime_1min (gauge - 1-minute moving average)</li>
<li>numActiveConnections.count</li>
<li>numRegisteredConnections.count</li>
<li>numCaughtExceptions.count</li>
<li>openBlockRequestLatencyMillis (histogram)</li>
<li>registerExecutorRequestLatencyMillis (histogram)</li>
<li>registeredExecutorsSize</li>
<li>shuffle-server.usedDirectMemory</li>
<li>
<p>shuffle-server.usedHeapMemory</p>
</li>
<li><strong>note:</strong> the metrics below apply when the server side configuration
<code class="language-plaintext highlighter-rouge">spark.shuffle.push.server.mergedShuffleFileManagerImpl</code> is set to
<code class="language-plaintext highlighter-rouge">org.apache.spark.network.shuffle.MergedShuffleFileManager</code> for Push-Based Shuffle</li>
<li>blockBytesWritten - size of the pushed block data written to file in bytes</li>
<li>blockAppendCollisions - number of shuffle push blocks collided in shuffle services
as another block for the same reduce partition were being written</li>
<li>lateBlockPushes - number of shuffle push blocks that are received in shuffle service
after the specific shuffle merge has been finalized</li>
<li>deferredBlocks - number of the current deferred block parts buffered in memory</li>
<li>deferredBlockBytes - size of the current deferred block parts buffered in memory</li>
<li>staleBlockPushes - number of stale shuffle block push requests</li>
<li>ignoredBlockBytes - size of the pushed block data that was transferred to ESS, but ignored.
The pushed block data are considered as ignored when: 1. it was received after the shuffle
was finalized; 2. when a push request is for a duplicate block; 3. ESS was unable to write the block.</li>
</ul>
<h1 id="advanced-instrumentation">Advanced Instrumentation</h1>
<p>Several external tools can be used to help profile the performance of Spark jobs:</p>
<ul>
<li>Cluster-wide monitoring tools, such as <a href="http://ganglia.sourceforge.net/">Ganglia</a>, can provide
insight into overall cluster utilization and resource bottlenecks. For instance, a Ganglia
dashboard can quickly reveal whether a particular workload is disk bound, network bound, or
CPU bound.</li>
<li>OS profiling tools such as <a href="http://dag.wieers.com/home-made/dstat/">dstat</a>,
<a href="http://linux.die.net/man/1/iostat">iostat</a>, and <a href="http://linux.die.net/man/1/iotop">iotop</a>
can provide fine-grained profiling on individual nodes.</li>
<li>JVM utilities such as <code class="language-plaintext highlighter-rouge">jstack</code> for providing stack traces, <code class="language-plaintext highlighter-rouge">jmap</code> for creating heap-dumps,
<code class="language-plaintext highlighter-rouge">jstat</code> for reporting time-series statistics and <code class="language-plaintext highlighter-rouge">jconsole</code> for visually exploring various JVM
properties are useful for those comfortable with JVM internals.</li>
</ul>
<p>Spark also provides a plugin API so that custom instrumentation code can be added to Spark
applications. There are two configuration keys available for loading plugins into Spark:</p>
<ul>
<li><code>spark.plugins</code></li>
<li><code>spark.plugins.defaultList</code></li>
</ul>
<p>Both take a comma-separated list of class names that implement the
<code>org.apache.spark.api.plugin.SparkPlugin</code> interface. The two names exist so that it&#8217;s
possible for one list to be placed in the Spark default config file, allowing users to
easily add other plugins from the command line without overwriting the config file&#8217;s list. Duplicate
plugins are ignored.</p>
</div>
<!-- /container -->
</div>
<script src="js/vendor/jquery-3.5.1.min.js"></script>
<script src="js/vendor/bootstrap.bundle.min.js"></script>
<script src="js/vendor/anchor.min.js"></script>
<script src="js/main.js"></script>
<script type="text/javascript" src="js/vendor/docsearch.min.js"></script>
<script type="text/javascript">
// DocSearch is entirely free and automated. DocSearch is built in two parts:
// 1. a crawler which we run on our own infrastructure every 24 hours. It follows every link
// in your website and extract content from every page it traverses. It then pushes this
// content to an Algolia index.
// 2. a JavaScript snippet to be inserted in your website that will bind this Algolia index
// to your search input and display its results in a dropdown UI. If you want to find more
// details on how works DocSearch, check the docs of DocSearch.
docsearch({
apiKey: 'd62f962a82bc9abb53471cb7b89da35e',
appId: 'RAI69RXRSK',
indexName: 'apache_spark',
inputSelector: '#docsearch-input',
enhancedSearchInput: true,
algoliaOptions: {
'facetFilters': ["version:3.5.0"]
},
debug: false // Set debug to true if you want to inspect the dropdown
});
</script>
<!-- MathJax Section -->
<script type="text/x-mathjax-config">
MathJax.Hub.Config({
TeX: { equationNumbers: { autoNumber: "AMS" } }
});
</script>
<script>
// Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS.
// We could use "//cdn.mathjax...", but that won't support "file://".
(function(d, script) {
script = d.createElement('script');
script.type = 'text/javascript';
script.async = true;
script.onload = function(){
MathJax.Hub.Config({
tex2jax: {
inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
processEscapes: true,
skipTags: ['script', 'noscript', 'style', 'textarea', 'pre']
}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
'?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
</script>
</body>
</html>