This is an automated email from the ASF dual-hosted git repository. srowen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 5fd28e8 [SPARK-26890][DOC] Add list of available Dropwizard metrics in Spark and add additional configuration details to the monitoring documentation 5fd28e8 is described below commit 5fd28e8f5c319492db4ef1e2c0a5a77f85ac029b Author: Luca Canali <luca.can...@cern.ch> AuthorDate: Wed Feb 27 10:07:15 2019 -0600 [SPARK-26890][DOC] Add list of available Dropwizard metrics in Spark and add additional configuration details to the monitoring documentation ## What changes were proposed in this pull request? This PR proposes to extend the documentation of the Spark metrics system in the monitoring guide. In particular by: - adding a list of the available metrics grouped per component instance - adding information on configuration parameters that can be used to configure the metrics system in alternative to the metrics.properties file - adding information on the configuration parameters needed to enable certain metrics - it also propose to add an example of Graphite sink configuration in metrics.properties.template Closes #23798 from LucaCanali/metricsDocUpdate. Authored-by: Luca Canali <luca.can...@cern.ch> Signed-off-by: Sean Owen <sean.o...@databricks.com> --- conf/metrics.properties.template | 14 ++- docs/monitoring.md | 256 ++++++++++++++++++++++++++++++++++++++- 2 files changed, 267 insertions(+), 3 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 4c008a1..23407e1 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -64,6 +64,10 @@ # "/metrics/applications/json" endpoints can be sent separately to get # metrics snapshots of the master instance and applications. This # MetricsServlet does not have to be configured. +# 6. The metrics system can also be configured using Spark configuration +# parameters. The relevant parameter names are formed by adding the +# prefix "spark.metrics.conf." to the configuration entries detailed in +# this file (see examples below). ## List of available common sources and their properties. @@ -172,6 +176,14 @@ # Unit of the polling period for the Slf4jSink #*.sink.slf4j.unit=minutes +# Example configuration for Graphite sink +#*.sink.graphite.class=org.apache.spark.metrics.sink.GraphiteSink +#*.sink.graphite.host=<graphiteEndPoint_hostName> +#*.sink.graphite.port=<listening_port> +#*.sink.graphite.period=10 +#*.sink.graphite.unit=seconds +#*.sink.graphite.prefix=<optional_value> + # Enable JvmSource for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource @@ -179,4 +191,4 @@ #driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource -#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource +#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource \ No newline at end of file diff --git a/docs/monitoring.md b/docs/monitoring.md index 726fb5c..a92dd6f 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -633,9 +633,13 @@ keep the paths consistent in both modes. Spark has a configurable metrics system based on the [Dropwizard Metrics Library](http://metrics.dropwizard.io/). This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV -files. The metrics system is configured via a configuration file that Spark expects to be present +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 `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the `spark.metrics.conf` [configuration property](configuration.html#spark-properties). +Instead of using the configuration file, a set of configuration parameters with prefix +`spark.metrics.conf.` can be used. By default, the root namespace used for driver or executor metrics is the value of `spark.app.id`. 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 @@ -684,9 +688,257 @@ code in your Spark package**_. For sbt users, set the the `-Pspark-ganglia-lgpl` profile. In addition to modifying the cluster's Spark build user applications will need to link to the `spark-ganglia-lgpl` artifact. -The syntax of the metrics configuration file is defined in an example configuration file, +The syntax of the metrics configuration file and the parameters available for each sink are defined +in an example configuration file, `$SPARK_HOME/conf/metrics.properties.template`. +When using Spark configuration parameters instead of the metrics configuration file, the relevant +parameter names are composed by the prefix `spark.metrics.conf.` followed by the configuration +details, i.e. the parameters take the following form: +`spark.metrics.conf.[instance|*].sink.[sink_name].[parameter_name]`. +This example shows a list of Spark configuration parameters for a Graphite sink: +``` +"spark.metrics.conf.*.sink.graphite.class"="org.apache.spark.metrics.sink.GraphiteSink" +"spark.metrics.conf.*.sink.graphite.host"="graphiteEndPoint_hostName>" +"spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port> +"spark.metrics.conf.*.sink.graphite.period"=10 +"spark.metrics.conf.*.sink.graphite.unit"=seconds +"spark.metrics.conf.*.sink.graphite.prefix"="optional_prefix" +``` + +Default values of the Spark metrics configuration are as follows: +``` +"*.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" +``` + +Additional sources can be configured using the metrics configuration file or the configuration +parameter `spark.metrics.conf.[component_name].source.jvm.class=[source_name]`. At present the +JVM source is the only available optional source. For example the following configuration parameter +activates the JVM source: +`"spark.metrics.conf.*.source.jvm.class"="org.apache.spark.metrics.source.JvmSource"` + +## List of available metrics providers + +Metrics used by Spark are of multiple types: gauge, counter, histogram, meter and timer, +see [Dropwizard library documentation for details](https://metrics.dropwizard.io/3.1.0/getting-started/). +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 `.count` 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. + +### Component instance = Driver +This is the component with the largest amount of instrumented metrics + +- namespace=BlockManager + - disk.diskSpaceUsed_MB + - memory.maxMem_MB + - memory.maxOffHeapMem_MB + - memory.maxOnHeapMem_MB + - memory.memUsed_MB + - memory.offHeapMemUsed_MB + - memory.onHeapMemUsed_MB + - memory.remainingMem_MB + - memory.remainingOffHeapMem_MB + - memory.remainingOnHeapMem_MB + +- namespace=HiveExternalCatalog + - fileCacheHits.count + - filesDiscovered.count + - hiveClientCalls.count + - parallelListingJobCount.count + - partitionsFetched.count + +- namespace=CodeGenerator + - compilationTime (histogram) + - generatedClassSize (histogram) + - generatedMethodSize (histogram) + - hiveClientCalls.count + - sourceCodeSize (histogram) + +- namespace=DAGScheduler + - job.activeJobs + - job.allJobs + - messageProcessingTime (timer) + - stage.failedStages + - stage.runningStages + - stage.waitingStages + +- namespace=LiveListenerBus + - listenerProcessingTime.org.apache.spark.HeartbeatReceiver (timer) + - listenerProcessingTime.org.apache.spark.scheduler.EventLoggingListener (timer) + - listenerProcessingTime.org.apache.spark.status.AppStatusListener (timer) + - numEventsPosted.count + - queue.appStatus.listenerProcessingTime (timer) + - queue.appStatus.numDroppedEvents.count + - queue.appStatus.size + - queue.eventLog.listenerProcessingTime (timer) + - queue.eventLog.numDroppedEvents.count + - queue.eventLog.size + - queue.executorManagement.listenerProcessingTime (timer) + +- namespace=appStatus (all metrics of type=counter) + - **note:** Introduced in Spark 3.0. Conditional to configuration parameter: + `spark.app.status.metrics.enabled=true` (default is false) + - stages.failedStages.count + - stages.skippedStages.count + - tasks.blackListedExecutors.count + - tasks.completedTasks.count + - tasks.failedTasks.count + - tasks.killedTasks.count + - tasks.skippedTasks.count + - tasks.unblackListedExecutors.count + - jobs.succeededJobs + - jobs.failedJobs + - jobDuration + +- namespace=AccumulatorSource + - **note:** User-configurable sources to attach accumulators to metric system + - DoubleAccumulatorSource + - LongAccumulatorSource + +- namespace=spark.streaming + - **note** applies to Spark Structured Streaming only. Conditional to a configuration + parameter: `spark.sql.streaming.metricsEnabled=true` (default is false) + - eventTime-watermark + - inputRate-total + - latency + - processingRate-total + - states-rowsTotal + - states-usedBytes + +### Component instance = Executor +These metrics are exposed by Spark executors. Note, currently they are not available +when running in local mode. + +- namespace=executor (metrics are of type counter or gauge) + - bytesRead.count + - bytesWritten.count + - cpuTime.count + - deserializeCpuTime.count + - deserializeTime.count + - diskBytesSpilled.count + - filesystem.file.largeRead_ops + - filesystem.file.read_bytes + - filesystem.file.read_ops + - filesystem.file.write_bytes + - filesystem.file.write_ops + - filesystem.hdfs.largeRead_ops + - filesystem.hdfs.read_bytes + - filesystem.hdfs.read_ops + - filesystem.hdfs.write_bytes + - filesystem.hdfs.write_ops + - jvmCpuTime + - jvmGCTime.count + - memoryBytesSpilled.count + - recordsRead.count + - recordsWritten.count + - resultSerializationTime.count + - resultSize.count + - runTime.count + - shuffleBytesWritten.count + - shuffleFetchWaitTime.count + - shuffleLocalBlocksFetched.count + - shuffleLocalBytesRead.count + - shuffleRecordsRead.count + - shuffleRecordsWritten.count + - shuffleRemoteBlocksFetched.count + - shuffleRemoteBytesRead.count + - shuffleRemoteBytesReadToDisk.count + - shuffleTotalBytesRead.count + - shuffleWriteTime.count + - threadpool.activeTasks + - threadpool.completeTasks + - threadpool.currentPool_size + - threadpool.maxPool_size + +- namespace=NettyBlockTransfer + - shuffle-client.usedDirectMemory + - shuffle-client.usedHeapMemory + - shuffle-server.usedDirectMemory + - shuffle-server.usedHeapMemory + +- namespace=HiveExternalCatalog + - fileCacheHits.count + - filesDiscovered.count + - hiveClientCalls.count + - parallelListingJobCount.count + - partitionsFetched.count + +- namespace=CodeGenerator + - compilationTime (histogram) + - generatedClassSize (histogram) + - generatedMethodSize (histogram) + - hiveClientCalls.count + - sourceCodeSize (histogram) + +### Source = JVM Source +Notes: + - Activate this source by setting the relevant `metrics.properties` file entry or the + configuration parameter:`spark.metrics.conf.*.source.jvm.class=org.apache.spark.metrics.source.JvmSource` + - This source is available for driver and executor instances and is also available for other instances. + - This source provides information on JVM metrics using the + [Dropwizard/Codahale Metric Sets for JVM instrumentation](https://metrics.dropwizard.io/3.1.0/manual/jvm/) + and in particular the metric sets BufferPoolMetricSet, GarbageCollectorMetricSet and MemoryUsageGaugeSet. + +### Component instance = applicationMaster +Note: applies when running on YARN + +- numContainersPendingAllocate +- numExecutorsFailed +- numExecutorsRunning +- numLocalityAwareTasks +- numReleasedContainers + +### Component instance = mesos_cluster +Note: applies when running on mesos + +- waitingDrivers +- launchedDrivers +- retryDrivers + +### Component instance = master +Note: applies when running in Spark standalone as master + +- workers +- aliveWorkers +- apps +- waitingApps + +### Component instance = ApplicationSource +Note: applies when running in Spark standalone as master + +- status +- runtime_ms +- cores + +### Component instance = worker +Note: applies when running in Spark standalone as worker + +- executors +- coresUsed +- memUsed_MB +- coresFree +- memFree_MB + +## Component instance = shuffleService +Note: applies to the shuffle service + +- blockTransferRateBytes (meter) +- numActiveConnections.count +- numRegisteredConnections.count +- openBlockRequestLatencyMillis (histogram) +- registerExecutorRequestLatencyMillis (histogram) +- registeredExecutorsSize +- shuffle-server.usedDirectMemory +- shuffle-server.usedHeapMemory + # Advanced Instrumentation Several external tools can be used to help profile the performance of Spark jobs: --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org