cerndb / SparkPlugins

Code and examples of how to write and deploy Apache Spark Plugins. Spark plugins allow runnig custom code on the executors as they are initialized. This also allows extending the Spark metrics systems with user-provided monitoring probes.
Apache License 2.0
82 stars 14 forks source link
kubernetes monitoring scala spark

SparkPlugins

SparkPlugins CI Maven Central

Use Spark Plugins to extend Apache Spark with custom metrics and executors' startup actions.

Key Features

Contents

Resources

Author and contact: Luca.Canali@cern.ch

Implementation Notes:


Getting Started - Your First Spark Plugins

Demo and Basic Plugins

Plugin for integrating with Pyroscope

Grafana Pyroscope is a tool for continuous profiling and Flame Graph visualization. This plugin allows to integrate Apache Spark and Pyroscope. For details see:
How to profile Spark with Pyroscope

An example of how to put all the configuration together and start Spark on a cluster with Pyroscope Flame Graph continuous monitoring. Example:

  1. Start Pyroscope

  2. Spark Spark (spark-shell, PySpark, spark-submit

    bin/spark-shell --master yarn  \
    --packages ch.cern.sparkmeasure:spark-plugins_2.12:0.3,io.pyroscope:agent:0.13.0 \ # update to use the latest versions
    --conf spark.plugins=ch.cern.PyroscopePlugin \
    --conf spark.pyroscope.server="http://<myhostname>:5040" # match with the server and port used when starting Pyroscope

Spark configurations:
This plugin adds the following configurations:

  --conf spark.pyroscope.server - > default "http://localhost:4040", update to match the server name and port used by Pyroscope
  --conf spark.pyroscope.applicationName -> default spark.conf.get("spark.app.id")
  --conf spark.pyroscope.eventType -> default ITIMER, possible values ITIMER, CPU, WALL, ALLOC, LOCK

Example:
This is an example of how to use the configuration programmatically (using PySpark):

from pyspark.sql import SparkSession

# Get the Spark session
spark = (SparkSession.builder.
      appName("Instrumented app").master("yarn")
      .config("spark.executor.memory","16g")
      .config("spark.executor.cores","4")
      .config("spark.executor.instances", 2)
      .config("spark.jars.packages", "ch.cern.sparkmeasure:spark-plugins_2.12:0.3,io.pyroscope:agent:0.13.0")
      .config("spark.plugins", "ch.cern.PyroscopePlugin")
      .config("spark.pyroscope.server", "http://<myhostname>:5040")
      .getOrCreate()
    )

OS metrics instrumentation with cgroups, for Spark on Kubernetes


Plugins to collect I/O storage statistics for HDFS and Hadoop Compatible Filesystems

HDFS extended storage statistics

This Plugin measures HDFS extended statistics. In particular, it provides information on read locality and erasure coding usage (for HDFS 3.x).


Cloud filesystem storage statistics for Hadoop Compatible Filesystems

CloudFSMetrics

This Plugin provides I/O statistics for Cloud Filesystem metrics (for s3a, gs, wasbs, oci, root, and any other storage system exposed as a Hadoop Compatible Filesystem).

CloudFSMetrics27

This Plugin provides I/O statistics for Cloud Filesystem metrics (for s3a, gs, wasbs, oci, root, and any other storage system exposed as a Hadoop Compatible Filesystem). Use this for Spark built using Hadoop 2.7.


Experimental Plugins for I/O Time Instrumentation

This section details a few experimental Spark plugins used to expose metrics for I/O-time instrumentation of Spark workloads using Hadoop-compliant filesystems.
These plugins use instrumented experimental/custom versions of the Hadoop client API for HDFS and other Hadoop-Compliant File Systems.