Sparrow: Distributed, Low Latency Scheduling
Sparrow: Distributed, Low Latency Scheduling
2
constraints over job placement, such as per-task con-
straints (e.g. each task needs to be co-resident with in- We assume a single wave job model when we evalu-
put data) and per-job constraints (e.g., all tasks must be ate scheduling techniques because single wave jobs are
placed on machines with GPUs). This feature set is simi- most negatively affected by the approximations involved
lar to that of the Hadoop MapReduce scheduler [23] and in our distributed scheduling approach: even a single
the Spark [26] scheduler. delayed task affects the job’s response time. However,
Sparrow also handles multiwave jobs.
sume that each job runs as a single wave of tasks. In is 1/(1 − ρ ), where ρ represents load. Using the least loaded of d
real clusters, jobs may run as multiple waves of tasks choices, wait time in an initially empty system over the first T units
d i −d
when, for example, m is greater than the number of slots of time is upper bounded by ∑∞ i=1 ρ
d−1 + o(1) [14].
2 The omniscient scheduler uses a greedy scheduling algorithm
assigned to the user; for multiwave jobs, the scheduler based on complete information about which worker machines are busy.
can place some early tasks on machines with longer For each incoming job, the scheduler places the job’s tasks on idle
queueing delay without affecting job response time. workers, if any exist, and otherwise uses FIFO queueing.
3
4 probes
Worker Worker
Scheduler Task 1 Scheduler (d = 2)
Worker Worker
Job Scheduler Worker Job Scheduler Worker
Task 2
Scheduler Worker Scheduler Worker
…
Worker Worker
Scheduler Scheduler
Worker Worker 2
(a) Per-task sampling selects queues of length 1 and 3. (b) Batch sampling selects queues of length 1 and 2.
Figure 2: Placing a parallel, two-task job. Batch sampling outperforms per-task sampling because tasks are
placed in the least loaded of the entire batch of sampled queues.
350
Random information from the probes sent for all of a job’s tasks,
300 Per-Task
Response Time (ms)
Batch and places the job’s m tasks on the least loaded of all the
250 Batch+Late Binding worker machines probed. In the example shown in Fig-
200 Omniscient ure 2, per-task sampling places tasks in queues of length
150 1 and 3; batch sampling reduces the maximum queue
100 length to 2 by using both workers that were probed by
Task 2 with per-task sampling.
50
To schedule using batch sampling, a scheduler ran-
0 domly selects dm worker machines (for d ≥ 1). The
0 0.2 0.4 0.6 0.8 1
Load scheduler sends a probe to each of the dm workers; as
with per-task sampling, each worker replies with the
Figure 3: Comparison of scheduling techniques in a number of queued tasks. The scheduler places one of the
simulated cluster of 10,000 4-core machines running job’s m tasks on each of the m least loaded workers. Un-
100-task jobs. less otherwise specified, we use d = 2; we justify this
choice of d in §7.9.
As shown in Figure 3, batch sampling improves per-
ration.3 As shown in Figure 3, response time increases
formance compared to per-task sampling. At 80% load,
with increasing load, because schedulers have less suc-
batch sampling provides response times 0.73× those
cess finding free machines on which to place tasks. At
with per-task sampling. Nonetheless, response times
80% load, per-task sampling improves performance by
with batch sampling remain a factor of 1.92× worse than
over 3× compared to random placement, but still results
those provided by an omniscient scheduler.
in response times equal to over 2.6× those offered by a
omniscient scheduler.
3.4 Problems with sample-based schedul-
3.3 Batch sampling ing
Batch sampling improves on per-task sampling by shar- Sample-based techniques perform poorly at high load
ing information across all of the probes for a particular due to two problems. First, schedulers place tasks based
job. Batch sampling is similar to a technique recently on the queue length at worker nodes. However, queue
proposed in the context of storage systems [18]. With length provides only a coarse prediction of wait time.
per-task sampling, one pair of probes may have gotten Consider a case where the scheduler probes two work-
unlucky and sampled two heavily loaded machines (e.g., ers to place one task, one of which has two 50ms tasks
Task 1 in Figure 2(a)), while another pair may have got- queued and the other of which has one 300ms task
ten lucky and sampled two lightly loaded machines (e.g, queued. The scheduler will place the task in the queue
Task 2 in Figure 2(a)); one of the two lightly loaded ma- with only one task, even though that queue will result
chines will go unused. Batch sampling aggregates load in a 200ms longer wait time. While workers could re-
3 We use this distribution because it puts the most stress on our
ply with an estimate of task duration rather than queue
approximate, distributed scheduling technique. When tasks within a
length, accurately predicting task durations is notori-
job are of different duration, the shorter tasks can have longer wait ously difficult. Furthermore, almost all task duration es-
times without affecting job response time. timates would need to be accurate for such a technique
4
to be effective, because each job includes many parallel and task runtimes are comparable, late binding will not
tasks, all of which must be placed on machines with low present a worthwhile tradeoff.
wait time to ensure good performance.
Sampling also suffers from a race condition where 3.6 Proactive Cancellation
multiple schedulers concurrently place tasks on a worker
that appears lightly loaded [13]. Consider a case where When a scheduler has launched all of the tasks for a par-
two different schedulers probe the same idle worker ma- ticular job, it can handle remaining outstanding probes
chine, w, at the same time. Since w is idle, both sched- in one of two ways: it can proactively send a cancel-
ulers are likely to place a task on w; however, only one lation RPC to all workers with outstanding probes, or
of the two tasks placed on the worker will arrive in an it can wait for the workers to request a task and reply
empty queue. The queued task might have been placed to those requests with a message indicating that no un-
in a different queue had the corresponding scheduler launched tasks remain. We use our simulation to model
known that w was not going to be idle when the task the benefit of using proactive cancellation and find that
arrived. proactive cancellation reduces median response time by
6% at 95% cluster load. At a given load ρ , workers are
busy more than ρ of the time: they spend ρ proportion of
3.5 Late binding time executing tasks, but they spend additional time re-
Sparrow introduces late binding to solve the aforemen- questing tasks from schedulers. Using cancellation with
tioned problems. With late binding, workers do not re- 1ms network RTT, a probe ratio of 2, and with tasks that
ply immediately to probes and instead place a reserva- are an average of 100ms long reduces the time work-
tion for the task at the end of an internal work queue. ers spend busy by approximately 1%; because response
When this reservation reaches the front of the queue, the times approach infinity as load approaches 100%, the
worker sends an RPC to the scheduler that initiated the 1% reduction in time workers spend busy leads to a no-
probe requesting a task for the corresponding job. The ticeable reduction in response times. Cancellation leads
scheduler assigns the job’s tasks to the first m workers to additional RPCs if a worker receives a cancellation for
to reply, and replies to the remaining (d − 1)m workers a reservation after it has already requested a task for that
with a no-op signaling that all of the job’s tasks have reservation: at 95% load, cancellation leads to 2% ad-
been launched. In this manner, the scheduler guarantees ditional RPCs. We argue that the additional RPCs are a
that the tasks will be placed on the m probed workers worthwhile tradeoff for the improved performance, and
where they will be launched soonest. For exponentially- the full Sparrow implementation includes cancellation.
distributed task durations at 80% load, late binding pro- Cancellation helps more when the ratio of network de-
vides response times 0.55× those with batch sampling, lay to task duration increases, so will become more im-
bringing response time to within 5% (4ms) of an omni- portant as task durations decrease, and less important as
scient scheduler (as shown in Figure 3). network delay decreases.
The downside of late binding is that workers are
idle while they are sending an RPC to request a new
task from a scheduler. All current cluster schedulers 4 Scheduling Policies and Con-
we are aware of make this tradeoff: schedulers wait to straints
assign tasks until a worker signals that it has enough
free resources to launch the task. In our target set- Sparrow aims to support a small but useful set of poli-
ting, this tradeoff leads to a 2% efficiency loss com- cies within its decentralized framework. This section
pared to queueing tasks at worker machines. The frac- describes support for two types of popular scheduler
tion of time a worker spends idle while requesting tasks policies: constraints over where individual tasks are
is (d · RTT)/(t + d · RTT) (where d denotes the num- launched and inter-user isolation policies to govern the
ber of probes per task, RTT denotes the mean network relative performance of users when resources are con-
round trip time, and t denotes mean task service time). In tended.
our deployment on EC2 with an un-optimized network
stack, mean network round trip time was 1 millisecond.
4.1 Handling placement constraints
We expect that the shortest tasks will complete in 100ms
and that scheduler will use a probe ratio of no more than Sparrow handles two types of constraints, per-job and
2, leading to at most a 2% efficiency loss. For our tar- per-task constraints. Such constraints are commonly re-
get workload, this tradeoff is worthwhile, as illustrated quired in data-parallel frameworks, for instance, to run
by the results shown in Figure 3, which incorporate net- tasks on a machine that holds the task’s input data
work delays. In environments where network latencies on disk or in memory. As mentioned in §2, Sparrow
5
does not support many types of constraints (e.g., inter- policies: strict priorities and weighted fair sharing. These
job constraints) supported by some general-purpose re- policies mirror those offered by other schedulers, includ-
source managers. ing the Hadoop Map Reduce scheduler [25].
Per-job constraints (e.g., all tasks should be run on Many cluster sharing policies reduce to using strict
a worker with a GPU) are trivially handled at a Spar- priorities; Sparrow supports all such policies by main-
row scheduler. Sparrow randomly selects the dm candi- taining multiple queues on worker nodes. FIFO, earliest
date workers from the subset of workers that satisfy the deadline first, and shortest job first all reduce to assign-
constraint. Once the dm workers to probe are selected, ing a priority to each job, and running the highest pri-
scheduling proceeds as described previously. ority jobs first. For example, with earliest deadline first,
Sparrow also handles jobs with per-task constraints, jobs with earlier deadlines are assigned higher priority.
such as constraints that limit tasks to run on machines Cluster operators may also wish to directly assign pri-
where input data is located. Co-locating tasks with input orities; for example, to give production jobs high prior-
data typically reduces response time, because input data ity and ad-hoc jobs low priority. To support these poli-
does not need to be transferred over the network. For cies, Sparrow maintains one queue for each priority at
jobs with per-task constraints, each task may have a dif- each worker node. When resources become free, Spar-
ferent set of machines on which it can run, so Sparrow row responds to the reservation from the highest prior-
cannot aggregate information over all of the probes in ity non-empty queue. This mechanism trades simplicity
the job using batch sampling. Instead, Sparrow uses per- for accuracy: nodes need not use complex gossip proto-
task sampling, where the scheduler selects the two ma- cols to exchange information about jobs that are waiting
chines to probe for each task from the set of machines to be scheduled, but low priority jobs may run before
that the task is constrained to run on, along with late high priority jobs if a probe for a low priority job ar-
binding. rives at a node where no high priority jobs happen to
Sparrow implements a small optimization over per- be queued. We believe this is a worthwhile tradeoff: as
task sampling for jobs with per-task constraints. Rather shown in §7.8, this distributed mechanism provides good
than probing individually for each task, Sparrow shares performance for high priority users. Sparrow does not
information across tasks when possible. For example, currently support preemption when a high priority task
consider a case where task 0 is constrained to run in arrives at a machine running a lower priority task; we
machines A, B, and C, and task 1 is constrained to run leave exploration of preemption to future work.
on machines C, D, and E. Suppose the scheduler probed Sparrow can also enforce weighted fair shares. Each
machines A and B for task 0, which were heavily loaded, worker maintains a separate queue for each user, and
and probed machines C and D for task 1, which were performs weighted fair queuing [6] over those queues.
both idle. In this case, Sparrow will place task 0 on ma- This mechanism provides cluster-wide fair shares in ex-
chine C and task 1 on machine D, even though both ma- pectation: two users using the same worker will get
chines were selected to be probed for task 1. shares proportional to their weight, so by extension, two
Although Sparrow cannot use batch sampling for jobs users using the same set of machines will also be as-
with per-task constraints, our distributed approach still signed shares proportional to their weight. We choose
provides near-optimal response times for these jobs, be- this simple mechanism because more accurate mecha-
cause even a centralized scheduler has only a small num- nisms (e.g., Pisces [22]) add considerable complexity;
ber of choices for where to place each task. Jobs with as we demonstrate in §7.7, Sparrow’s simple mechanism
per-task constraints can still use late binding, so the provides near-perfect fair shares.
scheduler is guaranteed to place each task on whichever
of the two probed machines where the task will run
sooner. Storage layers like HDFS typically replicate data 5 Analysis
on three different machines, so tasks that read input data
will be constrained to run on one of three machines Before delving into our experimental evaluation, we ana-
where the input data is located. As a result, even an lytically show that batch sampling achieves near-optimal
ideal, omniscient scheduler would only have one addi- performance, regardless of the task duration distribu-
tional choice for where to place each task. tion, given some simplifying assumptions. Section 3
demonstrated that Sparrow performs well, but only un-
4.2 Resource allocation policies der one particular workload; this section generalizes
those results to all workloads. We also demonstrate that
Cluster schedulers seek to allocate resources accord- with per-task sampling, performance decreases expo-
ing to a specific policy when aggregate demand for re- nentially with the number of tasks in a job, making it
sources exceeds capacity. Sparrow supports two types of poorly suited for parallel workloads.
6
n Number of servers in the cluster 10 tasks/job 100 tasks/job
1
ρ
7
Application Node Application
Spark App X Spark Scheduler
Frontend Frontend … Frontend Frontend
submitR
equest()
Monitor Executor
Sparrow Scheduler Sparrow Scheduler
enqueueR reserve time
eservation(
)
queue time
Worker Worker Worker
Time get task
Sparrow Node Monitor Sparrow Node Monitor
… Sparrow Node Monitor getTask()
launchT
time
Spark App X Spark App X ask()
Executor Executor Executor Executor service
time
taskComplete()
taskComplete()
taskComplete()
Figure 6: Frameworks that use Sparrow are decom-
posed into frontends, which generate tasks, and ex- Figure 7: RPCs (parameters not shown) and timings
ecutors, which run tasks. Frameworks schedule jobs associated with launching a job. Sparrow’s external
by communicating with any one of a set of distributed interface is shown in bold text and internal RPCs are
Sparrow schedulers. Sparrow node monitors run on shown in grey text.
each worker machine and federate resource usage.
8
6.3 Fault tolerance
a TPC-H workload, which features heterogeneous an-
Because Sparrow schedulers do not have any logically alytics queries. We provide fine-grained tracing of the
centralized state, the failure of one scheduler does not af- overhead that Sparrow incurs and quantify its perfor-
fect the operation of other schedulers. Frameworks that mance in comparison with an ideal scheduler. Second,
were using the failed scheduler need to detect the failure we demonstrate Sparrow’s ability to handle scheduler
and connect to a backup scheduler. Sparrow includes a failures. Third, we evaluate Sparrow’s ability to isolate
Java client that handles failover between Sparrow sched- users from one another in accordance with cluster-wide
ulers. The client accepts a list of schedulers from the ap- scheduling policies. Finally, we perform a sensitivity
plication and connects to the first scheduler in the list. analysis of key parameters in Sparrow’s design.
The client sends a heartbeat message to the scheduler it
is using every 100ms to ensure that the scheduler is still
alive; if the scheduler has failed, the client connects to 7.1 Performance on TPC-H workload
the next scheduler in the list and triggers a callback at the
We measure Sparrow’s performance scheduling queries
application. This approach allows frameworks to decide
how to handle tasks that were in-flight during the sched- from the TPC-H decision support benchmark. The TPC-
uler failure. Some frameworks may choose to ignore H benchmark is representative of ad-hoc queries on busi-
failed tasks and proceed with a partial result; for Spark, ness data, which are a common use case for low-latency
the handler instantly relaunches any phases that were in- data parallel frameworks.
flight when the scheduler failed. Frameworks that elect Each TPC-H query is executed using Shark [24],
to re-launch tasks must ensure that tasks are idempotent, a large scale data analytics platform built on top of
because the task may have been partway through execu- Spark [26]. Shark queries are compiled into multiple
tion when the scheduler died. Sparrow does not attempt Spark stages that each trigger a scheduling request using
to learn about in-progress jobs that were launched by the Sparrow’s submitRequest() RPC. Tasks in the first
failed scheduler, and instead relies on applications to re- stage are constrained to run on one of three machines
launch such jobs. Because Sparrow is designed for short holding the task’s input data, while tasks in remaining
jobs, the simplicity benefit of this approach outweighs stages are unconstrained. The response time of a query
the efficiency loss from needing to restart jobs that were is the sum of the response times of each stage. Because
in the process of being scheduled by the failed scheduler. Shark is resource-intensive, we use EC2 high-memory
While Sparrow’s design allows for scheduler failures, quadruple extra large instances, which each have 8 cores
Sparrow does not provide any safeguards against rogue and 68.4GB of memory, and use 4 slots on each worker.
schedulers. A misbehaving scheduler could use a larger Ten different users launch random permutations of the
probe ratio to improve performance, at the expensive of TPC-H queries to sustain an average cluster load of 80%
other jobs. In trusted environments where schedulers are for a period of approximately 15 minutes. We report re-
run by a trusted entity (e.g., within a company), this sponse times from a 200 second period in the middle
should not be a problem; in more adversarial environ- of the experiment; during the 200 second period, Spar-
ments, schedulers may need to be authenticated and rate- row schedules over 20k jobs that make up 6.2k TPC-H
limited to prevent misbehaving schedulers from wasting queries. Each user runs queries on a distinct denormal-
resources. ized copy of the TPC-H dataset; each copy of the data set
Sparrow does not handle worker failures, as discussed is approximately 2GB (scale factor 2) and is broken into
in §8, nor does it handle the case where the entire clus- 33 partitions that are each triply replicated in memory.
ter fails. Because Sparrow does not persist scheduling The TPC-H query workload has four qualities repre-
state to disk, in the event that all machines in the clus- sentative of a real cluster workload. First, cluster utiliza-
ter fail (for example, due to a power loss event), all jobs tion fluctuates around the mean value of 80% depending
that were in progress will need to be restarted. As in the on whether the users are collectively in more resource-
case when a scheduler fails, the efficiency loss from this intensive or less resource-intensive stages. Second, the
approach is minimal because jobs are short. stages have different numbers of tasks: the first stage has
33 tasks, and subsequent stages have either 8 tasks (for
reduce-like stages that read shuffled data) or 1 task (for
7 Experimental Evaluation aggregation stages). Third, the duration of each stage is
non-uniform, varying from a few tens of milliseconds to
We evaluate Sparrow using a cluster composed of 100 several hundred. Finally, the queries have a mix of con-
worker machines and 10 schedulers running on Ama- strained and unconstrained scheduling requests: 6.2k re-
zon EC2. Unless otherwise specified, we use a probe quests are constrained (the first stage in each query) and
ratio of 2. First, we use Sparrow to schedule tasks for the remaining 14k requests are unconstrained.
9
Random Batch + late binding Reserve time Get task time
Per-task sampling Ideal Queue time Service time
Batch sampling 1
Cumulative Probability
4217 (med.) 5396 (med.) 7881 (med.)
4000 0.8
3500
Response Time (ms)
3000 0.6
2500
2000 0.4
1500
0.2
1000
500 0
0 1 10 100
q3 q4 q6 q12 Milliseconds
Figure 8: Response times for TPC-H queries using Figure 9: Latency distribution for each phase in the
different placement stategies. Whiskers depict 5th Sparrow scheduling algorithm.
and 95th percentiles; boxes depict median, 25th, and
535 219
75th percentiles. 140
Per-task
120 Sparrow
100
Delay (ms)
80
60
40
To evaluate Sparrow’s performance, we compare 20
Sparrow to an ideal scheduler that always places all tasks 0
with zero wait time, as described in §3.1. To compute the Constrained Stages Unconstrained Stages
ideal response time for a query, we compute the response Figure 10: Delay using both Sparrow and per-task
time for each stage if all of the tasks in the stage had been sampling, for both constrained and unconstrained
placed with zero wait time, and then sum the ideal re- Spark stages. Whiskers depict 5th and 95 percentiles;
sponse times for all stages in the query. Sparrow always boxes depict median, 25th, and 75th percentiles.
satisfies data locality constraints; because the ideal re-
sponse times are computed using the service times when
Sparrow executed the job, the ideal response time as-
7.2 Deconstructing performance
sumes data locality for all tasks. The ideal response time To understand the components of the delay that Spar-
does not include the time needed to send tasks to worker row adds relative to an ideal scheduler, we deconstruct
machines, nor does it include queueing that is inevitable Sparrow scheduling latency in Figure 9. Each line cor-
during utilization bursts, making it a conservative lower responds to one of the phases of the Sparrow schedul-
bound on the response time attainable with a centralized ing algorithm depicted in Figure 7. The reserve time
scheduler. and queue times are unique to Sparrow—a centralized
Figure 8 demonstrates that Sparrow outperforms al- scheduler might be able to reduce these times to zero.
ternate techniques and provides response times within However, the get task time is unavoidable: no matter the
12% of an ideal scheduler. Compared to randomly as- scheduling algorithm, the scheduler will need to ship the
signing tasks to workers, Sparrow (batch sampling with task to the worker machine.
late binding) reduces median query response time by 4–
8× and reduces 95th percentile response time by over 7.3 How do task constraints affect perfor-
10×. Sparrow also reduces response time compared to
mance?
per-task sampling (a naı̈ve implementation based on the
power of two choices): batch sampling with late bind- Sparrow provides good absolute performance and im-
ing provides query response times an average of 0.8× proves over per-task sampling for both constrained and
those provided by per-task sampling. Ninety-fifth per- unconstrained tasks. Figure 10 depicts the delay for con-
centile response times drop by almost a factor of two strained and unconstrained stages in the TPC-H work-
with Sparrow, compared to per-task sampling. Late bind- load using both Sparrow and per-task sampling. Sparrow
ing reduces median query response time by an average schedules with a median of 7ms of delay for jobs with
of 14% compared to batch sampling alone. Sparrow also unconstrained tasks and a median of 14ms of delay for
provides good absolute performance: Sparrow provides jobs with constrained tasks; because Sparrow cannot ag-
median response times just 12% higher than those pro- gregate information across the tasks in a job when tasks
vided by an ideal scheduler. are constrained, delay is longer. Nonetheless, even for
10
6000
constrained tasks, Sparrow provides a performance im- 7.6 How does Sparrow compare to Spark’s
provement over per-task sampling due to its use of late
binding.
native, centralized scheduler?
Even in the relatively small, 100-node cluster in which
7.4 How do scheduler failures impact job we conducted our evaluation, Spark’s existing central-
response time? ized scheduler cannot provide high enough throughput
to support sub-second tasks.5 We use a synthetic work-
Sparrow provides automatic failover between schedulers load where each job is composed of 10 tasks that each
and can failover to a new scheduler in less than 120ms. sleep for a specified period of time, and measure job re-
Figure 11 plots the response time for ongoing TPC-H sponse time. Since all tasks in the job are the same du-
queries in an experiment parameterized as in §7.1, with ration, ideal job response time (if all tasks are launched
10 Shark frontends that submit queries. Each frontend immediately) is the duration of a single task. To stress
connects to a co-resident Sparrow scheduler but is ini- the schedulers, we use 8 slots on each machine (one per
tialized with a list of alternate schedulers to connect to in core). Figure 12 depicts job response time as a function
case of failure. At time t=20, we terminate the Sparrow of task duration. We fix cluster load at 80%, and vary
scheduler on node 1. The plot depicts response times for task submission rate to keep load constant as task du-
jobs launched from the Spark frontend on node 1, which ration decreases. For tasks longer than 2 seconds, Spar-
fails over to the scheduler on node 2. The plot also shows row and Spark’s native scheduler both provide near-ideal
response times for jobs launched from the Spark fron- response times. However, when tasks are shorter than
tend on node 2, which uses the scheduler on node 2 for 1355ms, Spark’s native scheduler cannot keep up with
the entire duration of the experiment. When the Sparrow the rate at which tasks are completing so jobs experience
scheduler on node 1 fails, it takes 100ms for the Spar- infinite queueing.
row client to detect the failure, less than 5ms to for the To ensure that Sparrow’s distributed scheduling is
Sparrow client to connect to the scheduler on node 2, necessary, we performed extensive profiling of the Spark
and less than 15ms for Spark to relaunch all outstand- scheduler to understand how much we could increase
ing tasks. Because of the speed at which failure recov- scheduling throughput with improved engineering. We
ery occurs, only 2 queries have tasks in flight during the did not find any one bottleneck in the Spark sched-
failure; these queries suffer some overhead. uler; instead, messaging overhead, virtual function call
overhead, and context switching lead to a best-case
7.5 Synthetic workload throughput (achievable when Spark is scheduling only
a single job) of approximately 1500 tasks per second.
The remaining sections evaluate Sparrow using a syn- Some of these factors could be mitigated, but at the ex-
thetic workload composed of jobs with constant dura- pense of code readability and understandability. A clus-
tion tasks. In this workload, ideal job completion time 5 For these experiments, we use Spark’s standalone mode, which
is always equal to task duration, which helps to isolate relies on a simple, centralized scheduler. Spark also allows for schedul-
the performance of Sparrow from application-layer vari- ing using Mesos; Mesos is more heavyweight and provides worse per-
ations in service time. As in previous experiments, these formance than standalone mode for short tasks.
11
400
Running Tasks 350 HP LP HP response LP response
300 load load time in ms time in ms
250 User 0 0.25 0 106 (111) N/A
200
150
User 1 0.25 0.25 108 (114) 108 (115)
100 0.25 0.5 110 (148) 110 (449)
50 0.25 0.75 136 (170) 40.2k (46.2k)
0 0.25 1.75 141 (226) 255k (270k)
0 10 20 30 40 50
Time (s) Table 3: Median and 95th percentile (shown in paren-
Figure 13: Cluster share used by two users that are theses) response times for a high priority (HP) and
each assigned equal shares of the cluster. User 0 sub- low priority (LP) user running jobs composed of 10
mits at a rate to utilize the entire cluster for the entire 100ms tasks in a 100-node cluster. Sparrow success-
experiment while user 1 adjusts its submission rate fully shields the high priority user from a low prior-
each 10 seconds. Sparrow assigns both users their ity user. When aggregate load is 1 or more, response
max-min fair share. time will grow to be unbounded for at least one user.
7.7 How well can Sparrow’s distributed 7.8 How much can low priority users hurt
fairness enforcement maintain fair response times for high priority users?
shares?
Table 3 demonstrates that Sparrow provides response
Figure 13 demonstrates that Sparrow’s distributed fair- times within 40% of an ideal scheduler for a high priority
ness mechanism enforces cluster-wide fair shares and user in the presence of a misbehaving low priority user.
quickly adapts to changing user demand. Users 0 and This experiment uses workers that each have 16 slots.
1 are both given equal shares in a cluster with 400 slots. The high priority user submits jobs at a rate to fill 25%
Unlike other experiments, we use 100 4-core EC2 ma- of the cluster, while the low priority user increases her
chines; Sparrow’s distributed enforcement works better submission rate to well beyond the capacity of the clus-
as the number of cores increases, so to avoid over stating ter. Without any isolation mechanisms, when the aggre-
performance, we evaluate it under the smallest number gate submission rate exceeds the cluster capacity, both
of cores we would expect in a cluster today. User 0 sub- users would experience infinite queueing. As described
mits at a rate to fully utilize the cluster for the entire in §4.2, Sparrow node monitors run all queued high pri-
duration of the experiment. User 1 changes her demand ority tasks before launching any low priority tasks, al-
every 10 seconds: she submits at a rate to consume 0%, lowing Sparrow to shield high priority users from mis-
25%, 50%, 25%, and finally 0% of the cluster’s available behaving low priority users. While Sparrow prevents the
slots. Under max-min fairness, each user is allocated her high priority user from experiencing infinite queueing
fair share of the cluster unless the user’s demand is less delay, the high priority user still experiences 40% worse
than her share, in which case the unused share is dis- response times when sharing with a demanding low pri-
tributed evenly amongst the remaining users. Thus, user ority user than when running alone on the cluster. This is
1’s max-min share for each 10-second interval is 0 con- because Sparrow does not use preemption: high priority
currently running tasks, 100 tasks, 200 tasks, 100 tasks, tasks may need to wait to be launched until low prior-
and finally 0 tasks; user 0’s max-min fair share is the re- ity tasks complete. In the worst case, this wait time may
maining resources. Sparrow’s fairness mechanism lacks be as long as the longest running low-priority task. Ex-
any central authority with a complete view of how many ploring the impact of preemption is a subject of future
tasks each user is running, leading to imperfect fairness work.
12
9279 (95th) 678 (med.), 4212 (95th)
300
574 (med.), 16 core, 50% long 4 cores, 50% long
4169 (95th) 4 cores, 10% long
Response Time (ms)
250
13
without adding significant complexity is a focus of fu- rely on centralized architectures. Among logically de-
ture work. Adding pre-emption, for example, would be a centralized schedulers, Sparrow is the first to sched-
simple way to mitigate the effects of low-priority users’ ule all of a job’s tasks together, rather than scheduling
jobs on higher priority users. each task independently, which improves performance
Constraints Our current design does not handle inter- for parallel jobs.
job constraints (e.g. “the tasks for job A must not run on Dean’s work on reducing the latency tail in serving
racks with tasks for job B”). Supporting inter-job con- systems [5] is most similar to ours. He proposes using
straints across frontends is difficult to do without signif- hedged requests where the client sends each request to
icantly altering Sparrow’s design. two workers and cancels remaining outstanding requests
Gang scheduling Some applications require gang when the first result is received. He also describes tied
scheduling, a feature not implemented by Sparrow. Gang requests, where clients send each request to two servers,
scheduling is typically implemented using bin-packing but the servers communicate directly about the status of
algorithms that search for and reserve time slots in which the request: when one server begins executing the re-
an entire job can run. Because Sparrow queues tasks on quest, it cancels the counterpart. Both mechanisms are
several machines, it lacks a central point from which similar to Sparrow’s late binding, but target an envi-
to perform bin-packing. While Sparrow often places all ronment where each task needs to be scheduled inde-
jobs on entirely idle machines, this is not guaranteed, pendently (for data locality), so information cannot be
and deadlocks between multiple jobs that require gang shared across the tasks in a job.
scheduling may occur. Sparrow is not alone: many clus- Work on load sharing in distributed systems (e.g., [7])
ter schedulers do not support gang scheduling [8, 9, 16]. also uses randomized techniques similar to Sparrow’s.
Query-level policies Sparrow’s performance could be In load sharing systems, each processor both generates
improved by adding query-level scheduling policies. A and processes work; by default, work is processed where
user query (e.g., a SQL query executed using Shark) it is generated. Processors re-distribute queued tasks if
may be composed of many stages that are each exe- the number of tasks queued at a processor exceeds some
cuted using a separate Sparrow scheduling request; to threshold, using either receiver-initiated policies, where
optimize query response time, Sparrow should sched- lightly loaded processors request work from randomly
ule queries in FIFO order. Currently, Sparrow’s algo- selected other processors, or sender-initiated policies,
rithm attempts to schedule jobs in FIFO order; adding where heavily loaded processors offload work to ran-
query-level scheduling policies should improve end-to- domly selected recipients. Sparrow represents a combi-
end query performance. nation of sender-initiated and receiver-initiated policies:
Worker failures Handling worker failures is compli- schedulers (“senders”) initiate the assignment of tasks
cated by Sparrow’s distributed design, because when a to workers (“receivers”) by sending probes, but work-
worker fails, all schedulers with outstanding requests ers finalize the assignment by responding to probes and
at that worker must be informed. We envision handling requesting tasks as resources become available.
worker failures with a centralized state store that relies Projects that explore load balancing tasks in multi-
on occasional heartbeats to maintain a list of currently processor shared-memory architectures (e.g., [19]) echo
alive workers. The state store would periodically dissem- many of the design tradeoffs underlying our approach,
inate the list of live workers to all schedulers. Since the such as the need to avoid centralized scheduling points.
information stored in the state store would be soft state, They differ from our approach because they focus
it could easily be recreated in the event of a state store on a single machine where the majority of the ef-
failure. fort is spent determining when to reschedule processes
Dynamically adapting the probe ratio Sparrow amongst cores to balance load.
could potentially improve performance by dynamically Quincy [9] targets task-level scheduling in compute
adapting the probe ratio based on cluster load; however, clusters, similar to Sparrow. Quincy maps the schedul-
such an approach sacrifices some of the simplicity of ing problem onto a graph in order to compute an optimal
Sparrow’s current design. Exploring whether dynami- schedule that balances data locality, fairness, and starva-
cally changing the probe ratio would significantly in- tion freedom. Quincy’s graph solver supports more so-
crease performance is the subject of ongoing work. phisticated scheduling policies than Sparrow but takes
over a second to compute a scheduling assignment in
a 2500 node cluster, making it too slow for our target
9 Related Work workload.
In the realm of data analytics frameworks,
Scheduling in distributed systems has been extensively Dremel [12] achieves response times of seconds
studied in earlier work. Most existing cluster schedulers with extremely high fanout. Dremel uses a hierarchical
14
scheduler design whereby each query is decomposed and strict priorities. Experiments using a synthetic work-
into a serving tree; this approach exploits the inter- load demonstrate that Sparrow is resilient to different
nal structure of Dremel queries so is not generally probe ratios and distributions of task durations. In light
applicable. of these results, we believe that distributed scheduling
Many schedulers aim to allocate resources at coarse using Sparrow presents a viable alternative to central-
granularity, either because tasks tend to be long-running ized schedulers for low latency parallel workloads.
or because the cluster supports many applications
that each acquire some amount of resources and per-
form their own task-level scheduling (e.g., Mesos [8], 11 Acknowledgments
YARN [16], Omega [20]). These schedulers sacrifice re-
quest granularity in order to enforce complex schedul- We are indebted to Aurojit Panda for help with debug-
ing policies; as a result, they provide insufficient latency ging EC2 performance anomalies, Shivaram Venkatara-
and/or throughput for scheduling sub-second tasks. High man for insightful comments on several drafts of this pa-
performance computing schedulers fall into this cate- per and for help with Spark integration, Sameer Agarwal
gory: they optimize for large jobs with complex con- for help with running simulations, Satish Rao for help
straints, and target maximum throughput in the tens with theoretical models of the system, and Peter Bailis,
to hundreds of scheduling decisions per second (e.g., Ali Ghodsi, Adam Oliner, Sylvia Ratnasamy, and Colin
SLURM [10]). Similarly, Condor supports complex fea- Scott for helpful comments on earlier drafts of this paper.
tures including a rich constraint language, job check- We also thank our shepherd, John Wilkes, for helping to
pointing, and gang scheduling using a heavy-weight shape the final version of the paper. Finally, we thank
matchmaking process that results in maximum schedul- the reviewers from HotCloud 2012, OSDI 2012, NSDI
ing throughput of 10 to 100 jobs per second [4]. 2013, and SOSP 2013 for their helpful feedback.
In the theory literature, a substantial body of work This research is supported in part by a Hertz Founda-
analyzes the performance of the power of two choices tion Fellowship, the Department of Defense through the
load balancing technique, as summarized by Mitzen- National Defense Science & Engineering Graduate Fel-
macher [15]. To the best of our knowledge, no exist- lowship Program, NSF CISE Expeditions award CCF-
ing work explores performance for parallel jobs. Many 1139158, DARPA XData Award FA8750-12-2-0331, In-
existing analyses consider placing balls into bins, and tel via the Intel Science and Technology Center for
recent work [18] has generalized this to placing multi- Cloud Computing (ISTC-CC), and gifts from Amazon
ple balls concurrently into multiple bins. This analysis Web Services, Google, SAP, Cisco, Clearstory Data,
is not appropriate for a scheduling setting, because un- Cloudera, Ericsson, Facebook, FitWave, General Elec-
like bins, worker machines process tasks to empty their tric, Hortonworks, Huawei, Microsoft, NetApp, Oracle,
queue. Other work analyzes scheduling for single tasks; Samsung, Splunk, VMware, WANdisco and Yahoo!.
parallel jobs are fundamentally different because a par-
allel job cannot complete until the last of a large number
of tasks completes. References
Straggler mitigation techniques (e.g., Dolly [2],
LATE [27], Mantri [3]) focus on variation in task ex- [1] Apache Thrift. [Link]
ecution time (rather than task wait time) and are com- org.
plementary to Sparrow. For example, Mantri launches a
task on a second machine if the first version of the task [2] G. Ananthanarayanan, A. Ghodsi, S. Shenker, and
is progressing too slowly, a technique that could easily I. Stoica. Why Let Resources Idle? Aggressive
be used by Sparrow’s distributed schedulers. Cloning of Jobs with Dolly. In HotCloud, 2012.
15
[5] J. Dean and L. A. Barroso. The Tail at Scale. Com- [17] K. Ousterhout, A. Panda, J. Rosen, S. Venkatara-
munications of the ACM, 56(2), February 2013. man, R. Xin, S. Ratnasamy, S. Shenker, and I. Sto-
ica. The Case for Tiny Tasks in Compute Clusters.
[6] A. Demers, S. Keshav, and S. Shenker. Analysis
In Proc. HotOS, 2013.
and Simulation of a Fair Queueing Algorithm. In
Proc. SIGCOMM, 1989. [18] G. Park. A Generalization of Multiple Choice
Balls-into-Bins. In Proc. PODC, pages 297–298,
[7] D. L. Eager, E. D. Lazowska, and J. Zahor-
2011.
jan. Adaptive Load Sharing in Homogeneous Dis-
tributed Systems. IEEE Transactions on Software [19] L. Rudolph, M. Slivkin-Allalouf, and E. Upfal. A
Engineering, 1986. Simple Load Balancing Scheme for Task Alloca-
[8] B. Hindman, A. Konwinski, M. Zaharia, A. Gh- tion in Parallel Machines. In Proc. SPAA, 1991.
odsi, A. D. Joseph, R. Katz, S. Shenker, and I. Sto- [20] M. Schwarzkopf, A. Konwinski, M. Abd-El-
ica. Mesos: A Platform For Fine-Grained Resource Malek, and J. Wilkes. Omega: flexible, scalable
Sharing in the Data Center. In Proc. NSDI, 2011. schedulers for large compute clusters. In Proc. Eu-
[9] M. Isard, V. Prabhakaran, J. Currey, U. Wieder, roSys, 2013.
K. Talwar, and A. Goldberg. Quincy: Fair Schedul- [21] B. Sharma, V. Chudnovsky, J. L. Hellerstein, R. Ri-
ing for Distributed Computing Clusters. In Proc. faat, and C. R. Das. Modeling and Synthesizing
SOSP, 2009. Task Placement Constraints in Google Compute
[10] M. A. Jette, A. B. Yoo, and M. Grondona. Clusters. In Proc. SOCC, 2011.
SLURM: Simple Linux Utility for Resource Man-
[22] D. Shue, M. J. Freedman, and A. Shaikh. Per-
agement. In Proc. Job Scheduling Strategies for
formance Isolation and Fairness for Multi-Tenant
Parallel Processing, Lecture Notes in Computer
Cloud Storage. In Proc. OSDI, 2012.
Science, pages 44–60. Springer, 2003.
[23] T. White. Hadoop: The Definitive Guide. O’Reilly
[11] M. Kornacker and J. Erickson. Cloudera Impala:
Media, 2009.
Real Time Queries in Apache Hadoop, For Real.
[Link] [24] R. S. Xin, J. Rosen, M. Zaharia, M. J. Franklin,
2012/10/cloudera-impala-real- S. Shenker, and I. Stoica. Shark: SQL and Rich
time-queries-in-apache-hadoop- Analytics at Scale. In Proc. SIGMOD, 2013.
for-real/, October 2012.
[25] M. Zaharia, D. Borthakur, J. Sen Sarma,
[12] S. Melnik, A. Gubarev, J. J. Long, G. Romer, K. Elmeleegy, S. Shenker, and I. Stoica. Delay
S. Shivakumar, M. Tolton, and T. Vassilakis. Scheduling: A Simple Technique For Achieving
Dremel: Interactive Analysis of Web-Scale Locality and Fairness in Cluster Scheduling. In
Datasets. Proc. VLDB Endow., 2010. Proc. EuroSys, 2010.
[13] M. Mitzenmacher. How Useful is Old Informa- [26] M. Zaharia, M. Chowdhury, T. Das, A. Dave,
tion? volume 11, pages 6–20, 2000. J. Ma, M. McCauley, M. J. Franklin, S. Shenker,
[14] M. Mitzenmacher. The Power of Two Choices and I. Stoica. Resilient Distributed Datasets: A
in Randomized Load Balancing. IEEE Trans- Fault-Tolerant Abstraction for In-Memory Cluster
actions on Parallel and Distributed Computing, Computing. In Proc. NSDI, 2012.
12(10):1094–1104, 2001.
[27] M. Zaharia, A. Konwinski, A. D. Joseph, R. Katz,
[15] M. Mitzenmacher. The Power of Two Random and I. Stoica. Improving MapReduce Performance
Choices: A Survey of Techniques and Results. In in Heterogeneous Environments. In Proc. OSDI,
S. Rajasekaran, P. Pardalos, J. Reif, and J. Rolim, 2008.
editors, Handbook of Randomized Computing, vol-
ume 1, pages 255–312. Springer, 2001.
[16] A. C. Murthy. The Next Generation of Apache
MapReduce. [Link]
com/blogs/hadoop/next-generation-
[Link],
February 2012.
16