Stage-Level Job Pipelining in Apache Spark: A Deep Dive

How we improved TracksETL throughput by understanding Spark’s execution model

Introduction

Back in 2018, we migrated TracksETL (one of the largest ETL job) from Hive to Spark to handle our exploding event volumes. At that time, we processed around 2.28 billion events monthly. Fast forward to today, and we’re processing over 15 billion events — a 6x increase. Throughout this growth, our Spark-based ETL has continued to perform well, largely thanks to an optimization we implemented: stage-level job pipelining.

In this post, I’ll walk through the evolution of our job scheduling strategy, from naive sequential processing to a sophisticated stage-aware pipelining approach that maximizes cluster utilization.

Understanding the Problem

TracksETL processes event data by day partitions. Due to data volume, we can’t process all days simultaneously — each day becomes a separate Spark job within a single application. A typical job has three distinct stages:

  1. READ – Read from HDFS (etl_events, prod_useraliases, etc.). This is very fast as it’s all sequential reads and parallelism is limited to the number of files on HDFS.
  2. PROCESS – Process and deduplicate data. This is computationally expensive and entails shuffling large amounts of data to join and de-duplicate.
  3. WRITE – Write back to HDFS. Since we don’t want to write data into a large number of small files, output is coalesced and then written. This is the slowest stage and is I/O bound.

The WRITE stage is particularly problematic: it’s I/O bound and uses fewer tasks (~120) than our available cores (~300), leaving resources idle.

The Evolution of Our Scheduling Strategy

Iteration Zero: Sequential Processing

The simplest approach — process one day at a time:

Result: Works with minimal resources, but painfully slow. The timeline was pretty boring and it never made it to production.

Iteration One: Parallel Jobs with par

As each stage was already individually optimized, it was not possible to get this running faster without running multiple jobs in parallel and throwing more hardware at it. This can be done by triggering Spark Actions for independent jobs in parallel. In Scala, this would look like:

jobs.par.foreach(job => job.compute)

Here, jobs is a list of jobs to be computed and compute triggers one of the Spark actions. par converts the list to a parallel list and hence they are submitted to Spark in parallel. By default, it will run four jobs in parallel.

This is a reasonably straightforward way of running parallel jobs without introducing any code complexity. In fact, for most cases this should be enough.

Problem: Jobs run in batches. The entire batch must complete before the next starts. In the diagram above, you can see that Jobs 2 and 3 finished before Job 1, but no new job is submitted until all jobs in the first batch are finished. This leads to some executors sitting idle when only a few tasks from one of the WRITE stages remain.

Iteration Two: Job-Level Pipelining

The next thing to try was to start a job as soon as one of the jobs is over:

I never implemented this approach as there was an even better approach.

Iteration Three: Stage-Level Pipelining (The Final Solution)

While describing the jobs earlier, I mentioned that the first two stages were fast/optimized but the third stage was I/O bound and slow. A WRITE stage is usually broken into around 120 tasks while we have around 300 cores dedicated for the TracksETL application.

Also, since there is considerable variation in volume of events that we get every day, it often happens that the WRITE stage of various parallel jobs start at different times. During each batch, we end up in a state with only WRITE stages running which are painfully slow.

To alleviate this issue, we can start the next job as soon as one of the already running jobs starts their WRITE stage. This will slow down the existing job a bit but not considerably, while we gain from starting the new job much earlier by not waiting for the whole job to finish. The wall-clock time for all the WRITE stages is amortized over the execution of READ and PROCESS stages.

The key insight: don’t wait for jobs to finish — start new jobs when running jobs enter their WRITE stage. Since WRITE is I/O bound and uses fewer cores, we can overlap it with the CPU-intensive READ and PROCESS stages of new jobs.

Implementation Deep Dive

The implementation uses Spark’s StatusTracker API to monitor job and stage progress. Here’s how it works:

Architecture Overview

The diagram below shows the overall architecture of our stage-level pipelining implementation:

The system consists of three main components:

  1. BlockingQueue – Holds all partition groups (days) to be processed. Each day’s data becomes a work item in the queue.
  2. Scheduling Loop – The main control loop that polls the StatusTracker every 20 seconds to check the state of running jobs and decides whether to submit new work.
  3. Thread Pool – An ExecutionContext with a fixed number of worker threads that execute the actual Spark jobs. Each worker pulls from the queue and processes partitions independently.

The Scheduling Decision Algorithm

The flowchart below illustrates the decision logic that runs every 20 seconds:

The algorithm follows these steps:

  1. Check capacity – If the number of active jobs is less than our target concurrency (tasks), submit a new job immediately.
  2. Query job stages – If we’re at capacity, use StatusTracker to get the current stage of each running job.
  3. Identify final stages – For each running job, find its maximum stage ID (which corresponds to its WRITE stage).
  4. Check for overlap opportunity – If all running jobs are currently executing their final (WRITE) stage, it’s safe to submit a new job because WRITE is I/O bound and won’t compete for CPU resources.

This approach ensures we maintain high throughput without overloading the cluster during CPU-intensive stages

Core Implementation

val statusTracker = sparkSession.sparkContext.statusTracker

  val blockingQueue = new ArrayBlockingQueue[PartitionGroup](
    partitionGroups.size,
    false,
    partitionGroups.asJava
  )

  while (!blockingQueue.isEmpty) {
    val activeJobIds: Array[Int] = statusTracker.getActiveJobIds()

    val newJob: Option[Future[Option[Seq[Long]]]] =
      if (activeJobIds.length < tasks) {
        // Simple case: we have capacity
        Some(Future { processPartitions(blockingQueue.poll().partitions) })
      } else {
        // Check if running jobs are in their final (WRITE) stage
        val runningJobStatuses = Seq(JobExecutionStatus.RUNNING, JobExecutionStatus.UNKNOWN)

        val runningJobInfos: Array[Option[SparkJobInfo]] =
          activeJobIds
            .map(statusTracker.getJobInfo)
            .filter {
              case None    => true
              case Some(x) => runningJobStatuses.contains(x.status())
            }

        if (runningJobInfos.length <= tasks + 1) {
          // Get the maximum (final) stage ID for each running job
          val maxStageIdsForRunningJobs: Array[Int] =
            runningJobInfos.flatten
              .filter(_.status() == JobExecutionStatus.RUNNING)
              .map(_.stageIds().max)

          val runningStageIds: Array[Int] = statusTracker.getActiveStageIds()

          // If all jobs are on their final stage, submit another
          if (maxStageIdsForRunningJobs.diff(runningStageIds).isEmpty) {
            Some(Future { processPartitions(blockingQueue.poll().partitions) })
          } else {
            None
          }
        } else {
          None
        }
      }

    newJob.foreach(j => futures.add(j))
    Thread.sleep(20000) // Poll every 20 seconds
  }

  // Wait for all jobs to complete
  Await.result(Future.sequence(futures.asScala), Duration(2, TimeUnit.HOURS))

The Key Insight Explained

The magic happens in this condition:

maxStageIdsForRunningJobs.diff(runningStageIds).isEmpty

Let’s break it down with an example:

Example: 3 jobs running, each has stages [0, 1, 2] where stage 2 is WRITE

  • Job A: stages [10, 11, 12] → max = 12
  • Job B: stages [20, 21, 22] → max = 22
  • Job C: stages [30, 31, 32] → max = 32

So maxStageIdsForRunningJobs = [12, 22, 32]

Case 1: Jobs still in PROCESS stage

runningStageIds = [11, 21, 31]  (middle stages)
  diff = [12, 22, 32] - [11, 21, 31] = [12, 22, 32]  (not empty)
  → DON'T submit new job (would overload cluster)

Case 2: All jobs in WRITE stage

runningStageIds = [12, 22, 32]  (final stages)
  diff = [12, 22, 32] - [12, 22, 32] = []  (empty!)
  → SUBMIT new job (WRITE is I/O bound, has spare CPU capacity)

Trade-offs and Considerations

Advantages

BenefitDescription
Better utilizationI/O-bound WRITE stages overlap with CPU-bound READ/PROCESS
Reduced wall-clock timeWRITE time is amortized across multiple job starts
No job changes neededScheduling is external to job logic
Graceful fallbackSet jobSubmissionThreads <= 1 for sequential mode

Disadvantages

Trade-offDescription
Code complexityRequires understanding Spark internals
Polling overhead20-second intervals add some latency
Memory pressureMultiple concurrent jobs increase memory usage
Debugging difficultyInterleaved jobs can be harder to trace

When to Use This Pattern

Good fit:

  • Many independent partitions to process
  • Jobs have distinct stage boundaries (READ → PROCESS → WRITE)
  • Final stages are I/O bound with lower parallelism
  • Cluster can handle 2-3 concurrent jobs

Not needed:

  • Simple, short-running jobs
  • Jobs with uniform resource usage across stages
  • Memory-constrained environments

Conclusion

By understanding Spark’s execution model — particularly how jobs decompose into stages with different resource profiles — we can implement intelligent scheduling that keeps our cluster busy. The key insight is that not all stages are equal: I/O-bound final stages can share resources with CPU-bound early stages of new jobs.

This optimization has allowed TracksETL to scale from 2 billion to 15+ billion events while maintaining acceptable processing times. The additional code complexity is worth it for workloads at this scale.

The StatusTracker API is underutilized in the Spark ecosystem. If you’re running large batch jobs with predictable stage patterns, consider whether stage-aware scheduling could improve your throughput.

Leave a Reply

Your email address will not be published. Required fields are marked *