SparkMeasure is a tool and library designed for efficient analysis and troubleshooting of Apache Spark jobs. It focuses on easing the collection and examination of Spark metrics, making it a practical choice for both developers and data engineers. With sparkMeasure, users can obtain a clearer understanding of their Spark job performance, facilitating smoother and more reliable data processing operations.
- Interactive Troubleshooting: Ideal for real-time analysis of Spark workloads in notebooks and spark-shell/pyspark environments.
- Development & CI/CD Integration: Facilitates testing, measuring, and comparing execution metrics of Spark jobs under various configurations or code changes.
- Batch Job Analysis: With Flight Recorder mode sparkMeasure records and analyzes batch job metrics for thorough inspection.
- Monitoring Capabilities: Seamlessly integrates with external systems like InfluxDB, Apache Kafka, and Prometheus PushPushgateway for extensive monitoring.
- Educational Tool: Serves as a practical example of implementing Spark Listeners for the collection of detailed Spark task metrics.
- Language Compatibility: Fully supports Scala, Java, and Python, making it versatile for a wide range of Spark applications.
- Getting started with sparkMeasure
- Documentation and API reference
- Notes on Metrics
- Architecture diagram
- Concepts and FAQ
- TPCDS PySpark - A tool you can use run TPCDS with PySpark, instrumented with sparkMeasure
- Spark monitoring dashboard - A custom monitoring pipeline and dashboard for Spark
- Introductory course on Apache Spark
- Notes on Apache Spark
Main author and contact: [email protected]
Choose the sparkMeasure version for your environment:
- For Spark 3.x, please use the latest version
- For Spark 2.4 and 2.3, use version 0.19
- For Spark 2.1 and 2.2, use version 0.16
Examples:
-
Spark with Scala 2.12:
- Scala:
bin/spark-shell --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24
- Python:
bin/pyspark --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24
- note: you also need
pip install sparkmeasure
to get the Python wrapper API
- note: you also need
- Scala:
-
Spark with Scala 2.13:
- Scala:
bin/spark-shell --packages ch.cern.sparkmeasure:spark-measure_2.13:0.24
- Python:
bin/pyspark --packages ch.cern.sparkmeasure:spark-measure_2.13:0.24
- note:
pip install sparkmeasure
to get the Python wrapper API
- note:
- Scala:
-
Spark 2.4 and 2.3 with Scala 2.11:
- Scala:
bin/spark-shell --packages ch.cern.sparkmeasure:spark-measure_2.11:0.19
- Python:
bin/pyspark --packages ch.cern.sparkmeasure:spark-measure_2.11:0.19
- note:
pip install sparkmeasure==0.19
to get the Python wrapper API
- note:
- Scala:
-
Where to get sparkMeasure:
- sparkMeasure on Maven Central
- Jars in sparkMeasure's release notes
- Bleeding edge jars as artifacts in GitHub actions
- Build jars from master using sbt:
sbt +package
-
Some practical examples of how to set the configuration to use sparkMeasure with Spark
--packages ch.cern.sparkmeasure:spark-measure_2.12:0.24
--jars /path/to/spark-measure_2.12-0.24.jar
--jars https://github.com/LucaCanali/sparkMeasure/releases/download/v0.24/spark-measure_2.12-0.24.jar
--conf spark.driver.extraClassPath=/path/to/spark-measure_2.12-0.24.jar
-
Stage-level metrics from the command line:
# Scala CLI bin/spark-shell --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24 val stageMetrics = ch.cern.sparkmeasure.StageMetrics(spark) stageMetrics.runAndMeasure(spark.sql("select count(*) from range(1000) cross join range(1000) cross join range(1000)").show())
# Python CLI pip install sparkmeasure bin/pyspark --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24 from sparkmeasure import StageMetrics stagemetrics = StageMetrics(spark) stagemetrics.runandmeasure(globals(), 'spark.sql("select count(*) from range(1000) cross join range(1000) cross join range(1000)").show()')
The output should look like this:
+----------+
| count(1)|
+----------+
|1000000000|
+----------+
Time taken: 3833 ms
Scheduling mode = FIFO
Spark Context default degree of parallelism = 8
Aggregated Spark stage metrics:
numStages => 3
numTasks => 17
elapsedTime => 1112 (1 s)
stageDuration => 864 (0.9 s)
executorRunTime => 3358 (3 s)
executorCpuTime => 2168 (2 s)
executorDeserializeTime => 892 (0.9 s)
executorDeserializeCpuTime => 251 (0.3 s)
resultSerializationTime => 72 (72 ms)
jvmGCTime => 0 (0 ms)
shuffleFetchWaitTime => 0 (0 ms)
shuffleWriteTime => 36 (36 ms)
resultSize => 16295 (15.9 KB)
diskBytesSpilled => 0 (0 Bytes)
memoryBytesSpilled => 0 (0 Bytes)
peakExecutionMemory => 0
recordsRead => 2000
bytesRead => 0 (0 Bytes)
recordsWritten => 0
bytesWritten => 0 (0 Bytes)
shuffleRecordsRead => 8
shuffleTotalBlocksFetched => 8
shuffleLocalBlocksFetched => 8
shuffleRemoteBlocksFetched => 0
shuffleTotalBytesRead => 472 (472 Bytes)
shuffleLocalBytesRead => 472 (472 Bytes)
shuffleRemoteBytesRead => 0 (0 Bytes)
shuffleRemoteBytesReadToDisk => 0 (0 Bytes)
shuffleBytesWritten => 472 (472 Bytes)
shuffleRecordsWritten => 8
Average number of active tasks => 3.0
Stages and their duration:
Stage 0 duration => 355 (0.4 s)
Stage 1 duration => 411 (0.4 s)
Stage 3 duration => 98 (98 ms)
- Stage metrics collection mode has an optional memory report command
- this is new in sparkMeasure since version 0.21, it requires Spark versions 3.1 or higher
- note: this report makes use of per-stage memory (executor metrics) data which is sent by the executors at each heartbeat to the driver, there could be a small delay or the order of a few seconds between the end of the job and the time the last metrics value is received.
- If you receive the error message java.util.NoSuchElementException: key not found, retry running the report after waiting for a few seconds.
(scala)> stageMetrics.printMemoryReport
(python)> stagemetrics.print_memory_report()
Additional stage-level executor metrics (memory usasge info):
Stage 0 JVMHeapMemory maxVal bytes => 322888344 (307.9 MB)
Stage 0 OnHeapExecutionMemory maxVal bytes => 0 (0 Bytes)
Stage 1 JVMHeapMemory maxVal bytes => 322888344 (307.9 MB)
Stage 1 OnHeapExecutionMemory maxVal bytes => 0 (0 Bytes)
Stage 3 JVMHeapMemory maxVal bytes => 322888344 (307.9 MB)
Stage 3 OnHeapExecutionMemory maxVal bytes => 0 (0 Bytes)
- Task-level metrics from the command line:
- this is similar but slightly different from the example above as it collects metrics at the Task-level rather than Stage-level
# Scala CLI bin/spark-shell --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24 val taskMetrics = ch.cern.sparkmeasure.TaskMetrics(spark) taskMetrics.runAndMeasure(spark.sql("select count(*) from range(1000) cross join range(1000) cross join range(1000)").show())
# Python CLI pip install sparkmeasure bin/pyspark --packages ch.cern.sparkmeasure:spark-measure_2.12:0.24 from sparkmeasure import TaskMetrics taskmetrics = TaskMetrics(spark) taskmetrics.runandmeasure(globals(), 'spark.sql("select count(*) from range(1000) cross join range(1000) cross join range(1000)").show()')
Spark is instrumented with several metrics, collected at task execution, they are described in the documentation:
Some of the key metrics when looking at a sparkMeasure report are:
- elapsedTime: the time taken by the stage or task to complete (in millisec)
- executorRunTime: the time the executors spent running the task, (in millisec). Note this time is cumulative across all tasks executed by the executor.
- executorCpuTime: the time the executors spent running the task, (in millisec). Note this time is cumulative across all tasks executed by the executor.
- jvmGCTime: the time the executors spent in garbage collection, (in millisec).
- shuffle metrics: several metrics with details on the I/O and time spend on shuffle
- I/O metrics: details on the I/O (reads and writes). Note, currently there are no time-based metrics for I/O operations.
To learn more about hte metrics, I advise you set up your lab environment and run some tests to see the metrics in action. A good place to start with is TPCDS PySpark - A tool you can use run TPCDS with PySpark, instrumented with sparkMeasure
SparkMeasure is one tool for many different use cases, languages, and environments:
-
Interactive mode
Use sparkMeasure to collect and analyze Spark workload metrics in interactive mode when working with shell or notebook environments, such asspark-shell
(Scala),PySpark
(Python) and/or fromjupyter notebooks
. -
Batch and code instrumentation
Instrument your code with the sparkMeasure API, for collecting, saving, and analyzing Spark workload metrics data. Examples and how-to guides:- Instrument Spark-Python code
- Instrument Spark-Scala code
- See also Spark_CPU_memory_load_testkit as an example of how to use sparkMeasure to instrument Spark code for performance testing.
-
Flight Recorder mode:
SparkMeasure in flight recorder will collect metrics transparently, without any need for you to change your code.- Metrics can be saved to a file, locally, or to a Hadoop-compliant filesystem
- or you can write metrics in near-realtime to the followingsinks: InfluxDB, Apache Kafka, Prometheus PushPushgateway
- More details:
-
Additional documentation and examples:
- Presentations at Spark/Data+AI Summit:
- Blog articles:
- TODO list and known issues
- TPCDS-PySpark a tool for running the TPCDS benchmark workload with PySpark and instrumented with sparkMeasure
- The tool is based on the Spark Listener interface. Listeners transport Spark executor Task Metrics data from the executor to the driver. They are a standard part of Spark instrumentation, used by the Spark Web UI and History Server for example.
- The tool is built on multiple modules implemented as classes
- metrics collection and processing can be at the Stage-level or Task-level. The user chooses which mode to use with the API.
- metrics are can be buffered into memory for real-time reporting, or they can be dumped to an external system in the "flight recorder mode".
- supported external systems are File Systems supported by the Hadoop API, InfluxDB, Apache Kafka, Prometheus Pushgateway.
- Metrics are flattened and collected into local memory structures in the driver (ListBuffer of a custom case class).
- sparkMeasure in flight recorder mode when using one between the InfluxDB sink, Apache Kafka sink, and Prometheus Pushgateway sink, does not buffer, but rather writes the collected metrics directly
- Metrics processing:
- metrics can be aggregated into a report showing the cumulative values for each metric
- aggregated metrics can also be returned as a Scala Map or Python dictionary
- metrics can be converted into a Spark DataFrame for custom querying
- Metrics data and reports can be saved for offline analysis.
-
Why measuring performance with workload metrics instrumentation rather than just using execution time measurements?
- When measuring just the jobs' elapsed time, you treat your workload as "a black box" and most often this does
not allow you to understand the root causes of performance regression.
With workload metrics you can (attempt to) go further in understanding and perform root cause analysis, bottleneck identification, and resource usage measurement.
- When measuring just the jobs' elapsed time, you treat your workload as "a black box" and most often this does
not allow you to understand the root causes of performance regression.
-
What are Apache Spark task metrics and what can I use them for?
- Apache Spark measures several details of each task execution, including run time, CPU time, information on garbage collection time, shuffle metrics, and task I/O. See also Spark documentation for a description of the Spark Task Metrics
-
How is sparkMeasure different from Web UI/Spark History Server and EventLog?
- sparkMeasure uses the same ListenerBus infrastructure used to collect data for the Web UI and Spark EventLog.
- Spark collects metrics and other execution details and exposes them via the Web UI.
- Notably, Task execution metrics are also available through the REST API
- In addition, Spark writes all details of the task execution in the EventLog file
(see config of
spark.eventlog.enabled
andspark.eventLog.dir
) - The EventLog is used by the Spark History server + other tools and programs that can read and parse the EventLog file(s) for workload analysis and performance troubleshooting, see a proof-of-concept example of reading the EventLog with Spark SQL
- There are key differences that motivate this development:
- sparkMeasure can collect data at the stage completion-level, which is more lightweight than measuring all the tasks, in case you only need to compute aggregated performance metrics. When needed, sparkMeasure can also collect data at the task granularity level.
- sparkMeasure has an API that makes it simple to add instrumentation/performance measurements in notebooks and in application code for Scala, Java, and Python.
- sparkMeasure collects data in a flat structure, which makes it natural to use Spark SQL for workload data analysis/
- sparkMeasure can sink metrics data into external systems (Filesystem, InfluxDB, Apache Kafka, Prometheus Pushgateway)
- sparkMeasure uses the same ListenerBus infrastructure used to collect data for the Web UI and Spark EventLog.
-
What are known limitations and gotchas?
- sparkMeasure does not collect all the data available in the EventLog
- See also the TODO and issues doc
- The currently available Spark task metrics can give you precious quantitative information on resources used by the executors, however there do not allow to fully perform time-based analysis of the workload performance, notably they do not expose the time spent doing I/O or network traffic.
- Metrics are collected on the driver, which could become a bottleneck. This is an issues affecting tools based on Spark ListenerBus instrumentation, such as the Spark WebUI. In addition, note that sparkMeasure in the current version buffers all data in the driver memory. The notable exception is when using the Flight recorder mode with InfluxDB or Apache Kafka or Prometheus Pushgateway sink, in this case metrics are directly sent to InfluxDB/Kafka/Prometheus Pushgateway.
- Task metrics values are collected by sparkMeasure only for successfully executed tasks. Note that resources used by failed tasks are not collected in the current version. The notable exception is with the Flight recorder mode with InfluxDB or Apache Kafka or Prometheus Pushgateway sink.
- sparkMeasure collects and processes data in order of stage and/or task completion. This means that the metrics data is not available in real-time, but rather with a delay that depends on the workload and the size of the data. Moreover, performance data of jobs executing at the same time can be mixed. This can be a noticeable issue if you run workloads with many concurrent jobs.
- Task metrics are collected by Spark executors running on the JVM, resources utilized outside the JVM are currently not directly accounted for (notably the resources used when running Python code inside the python.daemon in the case of Python UDFs with PySpark).
-
When should I use Stage-level metrics and when should I use Task-level metrics?
- Use stage metrics whenever possible as they are much more lightweight. Collect metrics at the task granularity if you need the extra information, for example if you want to study effects of skew, long tails and task stragglers.
-
How can I save/sink the collected metrics?
- You can print metrics data and reports to standard output or save them to files, using a locally mounted filesystem or a Hadoop compliant filesystem (including HDFS). Additionally, you can sink metrics to external systems (such as Prometheus Pushgateway). The Flight Recorder mode can sink to InfluxDB, Apache Kafka or Prometheus Pushgateway.
-
How can I process metrics data?
- You can use Spark to read the saved metrics data and perform further post-processing and analysis. See the also Notes on metrics analysis.
-
How can I contribute to sparkMeasure?
- SparkMeasure has already profited from users submitting PR contributions. Additional contributions are welcome. See the TODO_and_issues list for a list of known issues and ideas on what you can contribute.