You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/02/25 06:19:55 UTC

[GitHub] HeartSaVioR commented on a change in pull request #23798: [SPARK-26890][DOC] Add list of available Dropwizard metrics in Spark and add additional configuration details to the monitoring documentation

HeartSaVioR commented on a change in pull request #23798: [SPARK-26890][DOC] Add list of available Dropwizard metrics in Spark and add additional configuration details to the monitoring documentation
URL: https://github.com/apache/spark/pull/23798#discussion_r259687648
 
 

 ##########
 File path: docs/monitoring.md
 ##########
 @@ -684,9 +688,256 @@ 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 configuration parameters instead of the metrics configuration file, the relevant
+parameter names have the following form:
+`spark.metrics.conf.[instance|*].sink.[sink_name].[parameter_name]`.
+This example shows a list of 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 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
 
 Review comment:
   nit: two spaces after `=`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org