monotasks: architecting for performance clarity in data...

17
Monotasks: Architecting for Performance Clarity in Data Analytics Frameworks Kay Ousterhout UC Berkeley Christopher Canel Carnegie Mellon University Sylvia Ratnasamy UC Berkeley Scott Shenker UC Berkeley, ICSI ABSTRACT In today’s data analytics frameworks, many users struggle to reason about the performance of their workloads. With- out an understanding of what factors are most important to performance, users can’t determine what configuration pa- rameters to set and what hardware to use to optimize runtime. This paper explores a system architecture designed to make it easy for users to reason about performance bottlenecks. Rather than breaking jobs into tasks that pipeline many re- sources, as in today’s frameworks, we propose breaking jobs into monotasks: units of work that each use a single resource. We demonstrate that explicitly separating the use of different resources simplifies reasoning about performance without sacrificing performance. Monotasks provide job completion times within 9% of Apache Spark for typical scenarios, and lead to a model for job completion time that predicts run- time under different hardware and software configurations with at most 28% error. Furthermore, separating the use of different resources allows for new optimizations to improve performance. ACM Reference Format: Kay Ousterhout, Christopher Canel, Sylvia Ratnasamy, and Scott Shenker. 2017. Monotasks: Architecting for Performance Clarity in Data Analytics Frameworks. In Proceedings of SOSP ’17, Shanghai, China, October 28, 2017, 17 pages. https://doi.org/10.1145/3132747.3132766 Work done while at UC Berkeley Permission to make digital or hard copies of all or part of this work for personal or classroom use is granted without fee provided that copies are not made or distributed for profit or commercial advantage and that copies bear this notice and the full citation on the first page. Copyrights for components of this work owned by others than the author(s) must be honored. Abstracting with credit is permitted. To copy otherwise, or republish, to post on servers or to redistribute to lists, requires prior specific permission and/or a fee. Request permissions from [email protected]. SOSP ’17, October 28, 2017, Shanghai, China © 2017 Copyright held by the owner/author(s). Publication rights licensed to Association for Computing Machinery. ACM ISBN 978-1-4503-5085-3/17/10. . . $15.00 https://doi.org/10.1145/3132747.3132766 1 INTRODUCTION Users often spend significant energy trying to understand systems so that they can tune them for better performance. Performance questions that a user might ask include: What hardware should I run on? Is it worth it to get enough memory to cache on-disk data? How much will up- grading the network from 1Gbps to 10Gbps improve perfor- mance? What software configuration should I use? Should I store compressed or uncompressed data? How much work should be assigned concurrently to each machine? Why did my workload run so slowly? Is hardware degra- dation leading to poor performance? Is performance affected by contention from other users? Effectively answering the above questions can lead to sig- nificant performance improvements; for example, Venkatara- man et al. demonstrated that selecting an appropriate cloud instance type could improve performance by 1.9without increasing cost [32]. Yet answering these questions in exist- ing systems remains difficult. Moreover, expending signifi- cant effort to answer these questions once is not sufficient: users must continuously re-evaluate as software optimiza- tions, hardware improvements, and workload changes shift the bottleneck. Existing approaches have added performance visibility as an afterthought, e.g., by adding instrumentation to existing systems [4, 7, 11, 25]. We argue that architecting for perfor- mance clarity – making it easy to understand where bottle- necks lie and the performance implications of various system changes – should be an integral part of system design. Sys- tems that simplify reasoning about performance enable users to determine what configuration parameters to set and what hardware to use to optimize runtime. To provide performance clarity, we propose building sys- tems in which the basic unit of scheduling consumes only one resource. In the remainder of this paper, we apply this princi- ple to large-scale data analytics frameworks. We present an architecture that decomposes data analytics jobs into mono- tasks that each use exactly one of CPU, disk, or network. Each resource has a dedicated scheduler that schedules the

Upload: others

Post on 06-Aug-2021

2 views

Category:

Documents


0 download

TRANSCRIPT

Page 1: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity in Data

Analytics Frameworks

Kay OusterhoutUC Berkeley

Christopher Canel⇤Carnegie Mellon University

Sylvia RatnasamyUC Berkeley

Scott ShenkerUC Berkeley, ICSI

ABSTRACTIn today’s data analytics frameworks, many users struggleto reason about the performance of their workloads. With-out an understanding of what factors are most important toperformance, users can’t determine what configuration pa-rameters to set and what hardware to use to optimize runtime.This paper explores a system architecture designed to makeit easy for users to reason about performance bottlenecks.Rather than breaking jobs into tasks that pipeline many re-sources, as in today’s frameworks, we propose breaking jobsinto monotasks: units of work that each use a single resource.We demonstrate that explicitly separating the use of differentresources simplifies reasoning about performance withoutsacrificing performance. Monotasks provide job completiontimes within 9% of Apache Spark for typical scenarios, andlead to a model for job completion time that predicts run-time under different hardware and software configurationswith at most 28% error. Furthermore, separating the use ofdifferent resources allows for new optimizations to improveperformance.

ACM Reference Format:Kay Ousterhout, Christopher Canel, Sylvia Ratnasamy, and ScottShenker. 2017. Monotasks: Architecting for Performance Clarity inData Analytics Frameworks. In Proceedings of SOSP ’17, Shanghai,China, October 28, 2017, 17 pages.https://doi.org/10.1145/3132747.3132766

⇤Work done while at UC Berkeley

Permission to make digital or hard copies of all or part of this work forpersonal or classroom use is granted without fee provided that copies are notmade or distributed for profit or commercial advantage and that copies bearthis notice and the full citation on the first page. Copyrights for componentsof this work owned by others than the author(s) must be honored. Abstractingwith credit is permitted. To copy otherwise, or republish, to post on servers orto redistribute to lists, requires prior specific permission and/or a fee. Requestpermissions from [email protected] ’17, October 28, 2017, Shanghai, China© 2017 Copyright held by the owner/author(s). Publication rights licensed toAssociation for Computing Machinery.ACM ISBN 978-1-4503-5085-3/17/10. . . $15.00https://doi.org/10.1145/3132747.3132766

1 INTRODUCTIONUsers often spend significant energy trying to understandsystems so that they can tune them for better performance.Performance questions that a user might ask include:

What hardware should I run on? Is it worth it to getenough memory to cache on-disk data? How much will up-grading the network from 1Gbps to 10Gbps improve perfor-mance?

What software configuration should I use? Should Istore compressed or uncompressed data? How much workshould be assigned concurrently to each machine?

Why did my workload run so slowly? Is hardware degra-dation leading to poor performance? Is performance affectedby contention from other users?

Effectively answering the above questions can lead to sig-nificant performance improvements; for example, Venkatara-man et al. demonstrated that selecting an appropriate cloudinstance type could improve performance by 1.9⇥ withoutincreasing cost [32]. Yet answering these questions in exist-ing systems remains difficult. Moreover, expending signifi-cant effort to answer these questions once is not sufficient:users must continuously re-evaluate as software optimiza-tions, hardware improvements, and workload changes shiftthe bottleneck.

Existing approaches have added performance visibility asan afterthought, e.g., by adding instrumentation to existingsystems [4, 7, 11, 25]. We argue that architecting for perfor-mance clarity – making it easy to understand where bottle-necks lie and the performance implications of various systemchanges – should be an integral part of system design. Sys-tems that simplify reasoning about performance enable usersto determine what configuration parameters to set and whathardware to use to optimize runtime.

To provide performance clarity, we propose building sys-tems in which the basic unit of scheduling consumes only oneresource. In the remainder of this paper, we apply this princi-ple to large-scale data analytics frameworks. We present anarchitecture that decomposes data analytics jobs into mono-tasks that each use exactly one of CPU, disk, or network.Each resource has a dedicated scheduler that schedules the

Page 2: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

spark.textFile(“hdfs://…”).flatMap(lambda l: l.split(“ “)).map(lambda w: (w, 1)).reduceByKey(lambda a, b: a + b).saveAsTextFile(“hdfs://…”)

Map stage Reduce stage

Reduce Task

Network: read remote shuffle data

CPU: deserialize data, combine counts, serialize result Disk: read local shuffle data; write result

Map Task

Disk: read input from HDFS; write shuffle data

CPU: deserialize input, perform flatMap and map, serialize shuffle output

Key:

Task:

CPU use:

Disk use:

Network use:

Figure 1: Execution of a Spark job, written using Spark’s Python API, that performs word count. The job is broken intotwo stages, and each stage is broken into parallel tasks (shown as white boxes that execute in four parallel slots) thateach pipeline use of different resources.

monotasks for that resource. This design contrasts with to-day’s frameworks, which distribute work over machines bydividing it into tasks that each parallelize the use of CPU,disk, and network.

Decoupling the use of different resources into monotaskssimplifies reasoning about performance. With current frame-works, the use of fine-grained pipelining to parallelize CPU,disk, and network results in ad-hoc resource use that canchange at fine time granularity. This makes it difficult to rea-son about a job’s bottleneck, because even a single task’sresource bottleneck can change on short time horizons. Fur-thermore, tasks may contend at the granularity of the pipelin-ing; e.g., when one task’s disk read blocks waiting for a diskwrite triggered by a different task. This contention is handledby the operating system, making it difficult for the frameworkto report how different factors contributed to task runtime.In contrast, each monotask consumes a single resource fully,without blocking on other resources, making it trivial to rea-son about the resource use of a monotask and, as a result,the resource use of the job as a whole. Controlling each re-source with a dedicated scheduler allows that scheduler tofully utilize the resource and queue monotasks to controlcontention.

We used monotasks to implement MonoSpark, an API-compatible version of Apache Spark that replaces currentmulti-resource tasks with monotasks. MonoSpark does notsacrifice performance, despite eliminating fine-grained re-source pipelining: on three benchmark workloads, MonoSparkprovides job completion times within 9% of Spark for typicalscenarios (§5). In some cases, MonoSpark outperforms Spark,because per-resource schedulers allow MonoSpark to avoidresource contention and under utilization that occur as a resultof the non-uniform resource use during the lifespan of Sparktasks.

Using monotasks to schedule resources leads to a sim-ple model for job completion time based on the runtimesof each type of monotask. We use the model to predict theruntime of MonoSpark workloads under different hardware

configurations (e.g., with a different number or type of diskdrives), different software configurations (with data storedde-serialized and in-memory, rather than on-disk), and witha combination of both hardware and software changes (§6).For most “what-if” questions, the monotasks model providesestimates within 28% of the actual job completion time; forexample, monotasks correctly predicts the 10⇥ reduction inruntime from migrating a workload to a 4⇥ larger cluster withSSDs instead of HDDs.

Using monotasks also makes bottleneck analysis trivial.Monotask runtimes can easily be used to determine the bottle-neck resource, and we demonstrate that the monotasks modelcan be used to replicate the findings of a recent researchpaper [25] that used extensive instrumentation to performbottleneck analysis in Spark (§6.5).

Finally, using monotasks leads to new opportunities forperformance optimizations. For example, MonoSpark can au-tomatically determine the ideal task concurrency, which usersare required to specify in existing frameworks. We illustrateone example where using monotasks to automatically deter-mine the ideal concurrency improves job completion time by30% relative to configuring a job’s resource use.

2 BACKGROUND2.1 Architecture of data analytics

frameworksThis paper focuses on the design of data analytics frame-works like MapReduce [12], Dryad [18], and Spark [36]that provide an API for performing computations acrosslarge, distributed datasets. These frameworks rely on a bulk-synchronous-parallel model where jobs are broken into stages,and each stage is broken into parallel tasks that execute in-dependently and may read shuffled data from earlier stages.Figure 1 shows an example job, expressed using Spark’sPython API, and illustrates how the job would be broken intoparallel tasks in two stages. Each task in a stage performsthe same computation, but on a different block of input data.

Page 3: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

��������������������

���� ���� ���� ���� ���� ���� ����

�����������

��������

�����

��� ������ ������

Figure 2: As a result of fine-grained pipelining, resourceutilization during Spark jobs is non-uniform. This exam-ple illustrates the utilization during a 30 second periodwhen 8 tasks were running concurrently, and when thebottleneck changes between CPU and disk.

These frameworks are widely used because they allow usersto express computation using a simple API, and the frame-work abstracts away the details of breaking the computationinto stages and tasks that can be run in parallel.

These frameworks use fine-grained pipelining to parallelizeuse of CPU, network, and disk within each task, as shown inFigure 1. In Spark, each task has a single thread that processesone record at a time: the map task, for example, reads onerecord from disk, computes on that record, and writes it backto disk. Pipelining is implemented in the background; forexample, disk writes typically are written to buffer cache, andthe operating system writes to disk asynchronously. Pipeliningis not expressed by users as part of the API. For example, inthe map stage in Figure 1, the user specifies the name of theinput file and the transformation to apply to each line of inputdata, and the framework controls how the file contents areread from disk and passed to the user-specified computation.Frameworks implement pipelining to make tasks completemore quickly by parallelizing resource use.

2.2 The challenge of reasoning aboutperformance

The fine-grained pipelining orchestrated by each task makesreasoning about performance difficult for three reasons:

Tasks have non-uniform resource use. A task’s resourceprofile may change at fine time granularity as different partsof the pipeline become a bottleneck. For example, the reducetask in Figure 1 bottlenecks on CPU, network, and disk atdifferent points during its execution. Figure 2 illustrates thiseffect during a thirty-second period of time when 8 concurrentSpark tasks were running on a machine. During this period,the resource utilization oscillates between being bottleneckedon CPU and being bottlenecked on one of the disks, as a resultof fine-grained changes in each task’s resource usage.

Concurrent tasks on a machine may contend. Each useof network, CPU, or disk may contend with other tasks run-ning on the same machine. For example, when multiple taskssimultaneously issue disk reads or writes for the same disk,those requests will contend and take longer to complete. This

occurs in Figure 2 at 920 seconds, when all eight concurrenttasks block waiting on requests from the two disks.

Resource use occurs outside the control of the analyticsframework. Resource use is often triggered by the operatingsystem. For example, data written to disk is typically writ-ten to the buffer cache. The operating system, and not theframework, will eventually flush the cache, and this writemay contend with later disk reads or writes.

Together, these three challenges make reasoning about per-formance difficult. Consider a user who asks a question like“how much more quickly would my job have run if it usedtwice as many disks”. To answer this question, she wouldneed to walk through a task’s execution at the level of detailof the pipelining shown in the example tasks in Figure 1. Foreach fine-grained use of network, CPU, and disk, the userwould need to determine whether the time for that resourceuse would change in the new scenario, factoring in how tim-ing would be affected by the resource use of other tasks onthe same machine – which would each need to be modeled atsimilarly fine time granularity. The complexity of this processexplains the lack of simple models for job completion time.

Existing approaches to reasoning about performance haveaddressed this challenge in two different ways. One approachhas focused on adding instrumentation to measure when tasksblock on different resources; this instrumentation can be usedto answer simple questions about resource bottlenecks, butcannot be used for more sophisticated what-if questions [25].A second approach treats the analytics framework as a blackbox, and uses machine learning techniques to build a new per-formance model for each workload by running the workload(or a representative subset of the workload) repeatedly underdifferent configurations (e.g., Ernest [32] and CherryPick [5]).These models can answer what-if questions, but require of-fline training for each new workload. The complexity of theseapproaches is mandated by the challenges of reasoning aboutperformance in current frameworks.

Given the importance of the ability to reason about per-formance and the challenges to doing so today, this paperexplores architectural approaches to providing performanceclarity. We ask: would a different system architecture make iteasy to reason about performance? And would such an archi-tecture require a simplistic approach that sacrifices fast jobcompletion times?

3 MONOTASKS ARCHITECTUREThis paper explores a system architecture that provides perfor-mance clarity by using single-resource units of work calledmonotasks. Our focus in this paper is to explore how systemscan provide performance clarity, and as a result we do notfocus on optimizing our implementation to maximize per-formance. Using monotasks enables many new scheduling

Page 4: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

time

4 tasks run concurrently

on one worker

Network read CPU

Disk write

Pipelined task (multitask)

(a) Execution as today’s multi-resource tasks.

Monotasks for the first multitask

time

Network monotasks CPU scheduler runs

one monotask per core

HDD scheduler runs one monotask per disk

(b) Execution as monotasks. Arrows represent dependencies.

Figure 3: Execution of eight multitasks on a single worker machine. With current frameworks, shown in (a), eachmultitask parallelizes reading data over the network, filtering some of the data (using the CPU), and writing the resultto disk. Using monotasks, shown in (b), each of today’s multitasks is decomposed into a DAG of monotasks, each ofwhich uses exactly one resource. Per-resource schedulers regulate access to each resource.

optimizations, and we leave exploration of such optimizationsto future work.

3.1 DesignThe monotasks design replaces the fine-grained pipeliningused in today’s tasks – henceforth referred to as multitasks –with statistical multiplexing across monotasks that each use asingle resource. The design is based on four principles:

Each monotask uses one resource. Jobs are decomposedinto units of work called monotasks that each use exactly oneof CPU, network, and disk. As a result, the resource use ofeach task is uniform and predictable.

Monotasks execute in isolation. To ensure that each mono-task can fully utilize the underlying resource, monotasks donot interact with or block on other monotasks during theirexecution.

Per-resource schedulers control contention. Each workermachine has a set of schedulers that are each responsible forscheduling monotasks on one resource. These resource sched-ulers are designed to run the minimum number of monotasksnecessary to keep the underlying resource fully utilized, andqueue remaining monotasks. For example, the CPU schedulerruns one monotask per CPU core. This design makes resourcecontention “visible” as the queue length for each resource.

Per-resource schedulers have complete control over eachresource. To ensure that the per-resource schedulers can con-trol contention for each resource, monotasks avoid optimiza-tions that involve the operating system triggering resourceuse. For example, disk monotasks flush all writes to disk, toavoid situations where the OS buffer cache contends withother disk monotasks.

Figure 3 compares execution of multitasks on a singleworker machine with current frameworks, shown in (a), withhow those multitasks would be decomposed into monotasksand executed by per-resource schedulers, shown in (b).

The principles above represent the core ideas underlyingmonotasks. A variety of architectures could support theseprinciples; for example, there are many approaches to sched-uling monotasks. The remainder of this section focuses on the

design decisions that we made to support monotasks specifi-cally in the context of Apache Spark; we refer to the resultingsystem as MonoSpark.

3.2 How are multitasks decomposed intomonotasks?

Decomposing jobs into monotasks does not require users towrite their jobs as a sequence of monotasks, and can be doneinternally by the framework without changing the existingAPI. This is possible for frameworks such as Spark becausethey provide a high-level API: users describe the locationof input data, the computation to perform on the data, andwhere to store output data, and Spark is responsible for im-plementing the necessary disk and network I/O. Because theframework is responsible for the details of how I/O occurs, itcan replace the existing fine-grained pipelining with mono-tasks without requiring changes to user code.

With MonoSpark, the decomposition of jobs into mono-tasks is performed on worker machines rather than by thecentral job scheduler. The job scheduler works in the sameway as in today’s frameworks: it decomposes jobs into par-allel multitasks that perform a computation on a single inputdata block, and that run on one machine. These multitasksare the same as tasks in current frameworks, and as in currentframeworks, they are assigned to workers based on data local-ity: if a multitask reads input data that is stored on disk, it willbe assigned to a worker machine that holds the input data.

Multitasks are decomposed into monotasks when they ar-rive on the worker machine. Figure 4 illustrates how jobs aredecomposed into monotasks using the word count job fromFigure 1 as an example. To read input data from HDFS, thejob uses the textFile function, and passes in the name of afile in HDFS. With both Spark and MonoSpark, the job sched-uler creates one multitask for each block of the file (HDFSbreaks files into blocks, and distributes the blocks over acluster of machines). Spark implements fine-grained pipelin-ing to read and compute on the block: each Spark multitaskpipelines reading the file block’s data from disk with com-putation on the data. MonoSpark instead creates a disk read

Page 5: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

spark.textFile(“hdfs://…”).flatMap(lambda l: l.split(“ “)).map(lambda w: (w, 1)).reduceByKey(lambda a, b: a + b).saveAsTextFile(“hdfs://…”)

Deserialize input, perform flatMap and map,

serialize shuffle output

Write shuffle data to local

disk

Read input as bytes

from HDFS

Map multitask(one pipelined map task in Spark)

Request remote

shuffle data Read shuffle data

from disk Send shuffle

data

Read shuffle data from local disk

Deserialize shuffle data,

combine counts, serialize result

Write result (as bytes) to

HDFS

Reduce multitask(one pipelined reduce task

in Spark)

Key:

Compute monotask:

Disk monotask:

Network monotask:

Dependency:

Map stage

Reduce stage

Executed on remote machines

Figure 4: Decomposition of a Spark job that performs word count (written using Spark’s Python API) into monotasks.The grey boxes show the two stages that the job would be broken into, and the white box within each stage illustrateshow one multitask in the stage is decomposed into monotasks. For brevity the figure omits a compute monotask at thebeginning of each multitask, which deserializes the task received from the scheduler and creates the DAG of monotasks,and a compute monotask at the end of each multitask, which serializes metrics about the task’s execution.

monotask, which reads all of the file block’s bytes from diskinto a serialized, in-memory buffer, followed by a computemonotask, which deserializes the bytes, emits a count of 1 foreach instance of each word, and serializes the resulting datainto a new in-memory buffer. The shuffle is similarly specifiedusing a high-level API: the function reduceByKey is usedto aggregate all of the counts for each word, which is doneusing a shuffle. Spark uses pipelining to parallelize readingshuffle data from disk and over the network with deserial-izing the data and performing the computation specified inthe reduceByKey call. MonoSpark implements the shuffleby breaking each reduce multitask into network monotasksthat each issue a request for shuffle data to remote machines.When remote machines receive the request, they create adisk read monotask to read all of the requested shuffle datainto memory, followed by a network monotask to send thedata back to the machine where the reduce multitask is ex-ecuting. Once all shuffle data has been received and placedin in-memory buffers, a compute monotask deserializes allof the shuffle data and performs the necessary computation.Other functions in Spark’s API are implemented in a similarmanner: Spark’s API accepts a high-level description of thecomputation and the data to operate on, but not details of howresources are used, so MonoSpark can transparently changefine-grained pipelining to monotasks.

Because using monotasks eliminates the use of fine-grainedpipelining to parallelize resource use, a single multitask willtake longer to complete with monotasks than with today’sframeworks. For example, for the map task in Figure 4, usingmonotasks serializes the resource use, causing the multitaskto take longer than when resource use is parallelized usingfine-grained pipelining (as in Figure 1). In order to avoidincreasing job completion time, jobs must be broken down

into enough multitasks to enable pipelining between indepen-dent monotasks in the same job. In other words, the mono-tasks design forgoes fine-grained pipelining within a singlemultitask and instead achieves high utilization through thestatistical multiplexing of monotasks. We evaluate the sensi-tivity of MonoSpark’s performance to the number of tasks ina job in §5.3, and discuss the resulting limitations on whenMonoSpark can be used effectively in §8.

3.3 Scheduling monotasks on each workerOn each worker, monotasks are scheduled using two lay-ers of schedulers. A top level scheduler, called the LocalDAG Scheduler, manages the directed acyclic graph (DAG)of monotasks for each multitask. The Local DAG Sched-uler tracks the dependencies for each monotask, and submitsmonotasks to the appropriate per-resource scheduler whenall dependencies have completed. For example, for the mapmultitask in Figure 4, the Local DAG Scheduler will waitto submit the compute monotask to the compute scheduleruntil the disk read monotask has completed. The Local DAGScheduler is necessary to ensure that monotasks can fullyutilize the underlying resource and do not block on othermonotasks during their execution.

The Local DAG Scheduler assigns monotasks to dedicated,per-resource schedulers that each seek to fully utilize theunderlying resource while minimizing contention:

CPU scheduler The CPU scheduler is straightforward: onemonotask can fully utilize one core, so the CPU schedulerruns one monotask per core and queues remaining monotasks.

Disk scheduler The hard disk scheduler runs one mono-task per disk, because running multiple concurrent monotasks

Page 6: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

reduces throughput due to seek time.1 Flash drives, on theother hand, can provide higher throughput when multipleoperations are outstanding. The flash scheduler exposes aconfiguration parameter that allows users to change the num-ber of concurrent flash monotasks based on the underlyinghardware. For the flash drives we used, we found that usingfour outstanding monotasks achieved nearly the maximumthroughput (results omitted for brevity).

Network scheduler Efficiently scheduling the network isthe most challenging, because scheduling network monotasksrequires coordination across machines. If a machine A startssending data to machine B, the transfer may contend withother flows originating at A, or other flows destined to B.Determining how to match sender demand and receiver band-width to optimize utilization is an NP-hard problem that hasbeen studied extensively. MonoSpark adopts a simple ap-proach where all scheduling occurs at the receiver, whichlimits its outstanding requests. We chose the number of out-standing requests to balance two objectives. Consider a ma-chine running multiple reduce multitasks that are each fetch-ing shuffle data from all machines where map multitasks wereexecuted. Issuing the requests for just one multitask at a timehurts utilization: the multitask could be waiting on data fromjust one (slow) remote machine, causing the receiving linkto be underutilized. On the other hand, issuing the requestsfor too many multitasks at a time also hurts performance. Be-cause the monotasks design relies on coarse-grained pipelin-ing between monotasks for different multitasks, jobs com-plete most quickly when all of the data for one multitask isreceived before using any of the receiver bandwidth for thenext multitask’s data. This way the compute monotask (toprocess received data) can be pipelined with the next multi-task’s network requests. To balance these two issues, we limitthe number of outstanding requests to those coming from fourmultitasks, based on an experimental parameter sweep. As inother parts of the monotasks design, more sophisticated sched-ulers (e.g., based on distributed matching between sendersand receivers, as in pHost [13] and iSlip [20]) are possible,and we leave exploration of these to future work.

Queueing monotasks When more monotasks are waitingfor a resource than can run concurrently, monotasks will bequeued. The queuing algorithm is important to maintaininghigh resource utilization when multitasks include multiplemonotasks that use the same resource. Consider multitasksthat are made up of three monotasks: a disk read monotask,a compute monotask, and a disk write monotask. If a queueof disk write monotasks accumulates (e.g., when disk is thebottleneck), as writes finish and new multitasks are assigned1When disk monotasks transfer a small amount of data, disk throughput canbe improved by running many parallel monotasks, so the disk scheduler canoptimize seek time by re-ordering monotasks. We leave exploration of this tofuture work.

to the machine, the read monotasks for the new multitaskswill be stuck in the long disk queue behind all of the writes.Because each compute monotask depends on a read monotaskcompleting first, the CPU will remain idle until all of thedisk writes have finished. After a burst of CPU use, the diskqueue will again build up with the disk writes, and futureCPU use will be delayed until all of the writes complete. Thiscycle will continue and harms utilization because it preventsCPU and disk from being used concurrently. Intuitively, tomaintain high utilization, the system needs to maintain apipeline of monotasks to execute on all resources. To solvethis problem, queues implement round robin over monotasksin different phases of the multitask DAG. For example, in theexample above, the disk scheduler would implement roundrobin between disk read and disk write monotasks.

3.4 How many multitasks should be assignedconcurrently to each machine?

The MonoSpark job scheduler works in the same way as theSpark job scheduler, with one exception: with MonoSpark,more multitasks need to be concurrently assigned to eachmachine to fully utilize the machine’s resources. ConsiderFigure 3 as an example: with current frameworks, four tasksrun concurrently (one on each core), whereas when jobs aredecomposed into monotasks, four multitasks can concurrentlybe running CPU monotasks, while additional multitasks canuse the disk and the network.

Before discussing how many multitasks to assign to eachmachine in MonoSpark, we note that the trade-offs in makingthis decision are different than in current frameworks. In cur-rent frameworks, the number of tasks concurrently assignedto each machine is intimately tied to resource use: schedulersmust navigate a trade-off between assigning more tasks toeach machine, to drive up utilization, and assigning fewertasks, to avoid contention. With the per-resource monotaskschedulers, on the other hand, each resource scheduler avoidscontention by queuing monotasks beyond the number that canrun efficiently on each resource. As a result, the job sched-uler does not need to limit the number of outstanding tasksto avoid contention, and the primary risk is under-utilizingresources by assigning too few tasks per machine.

To ensure that all resources can be fully utilized, MonoSparkassigns enough multitasks that all resources can have the max-imum allowed number of concurrent monotasks running, plusone additional monotask. For example, if a machine has fourCPU cores and one hard disk, the job scheduler will assignten concurrent multitasks: enough for four to have computemonotasks running, one to have a disk monotask running,four have monotasks running on the network, and one extra.The extra multitask exists for the purposes of the round-robinscheduling described in §3.3: without this extra monotask,

Page 7: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

one of the queues in the round-robin ordering can get skipped,because it is temporarily empty while a new multitask is beingrequested from the job scheduler.

The strategy emphasizes simplicity, and a more sophis-ticated scheduling strategy that accounted for the types ofmonotasks that make up each multitask and the current re-source queue lengths on each worker machine would likelyimprove performance. The goal of this paper is to explore theperformance of a relatively simple and unoptimized mono-tasks design, so we leave exploration of more complicatedscheduling techniques to future work.

3.5 How is memory access regulated?Breaking jobs into monotasks means that more memory isused on each worker: in Figure 4, for example, all of themap multitask’s data is read into memory before computa-tion begins, whereas with current frameworks, fine-grainedpipelining means that the data would be incrementally read,computed on, and written back to disk. This can result in ad-ditional garbage collection, which slows job completion time.This can also cause workers to run out of memory. Monotasksschedulers could prioritize monotasks based on the amount ofremaining memory; e.g., the disk scheduler could prioritizedisk write monotasks over read monotasks when memory iscontended, to clear data out of memory. This paper does notexplore strategies to regulate memory use; we discuss thislimitation in more detail in §8.

4 IMPLEMENTATIONThe previous section described the design of MonoSpark; thissection describes lower-level implementation details. MonoSparkuses monotasks to replace the task execution code in ApacheSpark [1, 36].2 MonoSpark is compatible with Spark’s publicAPI: if a developer has written an application on top of Spark,she can change to using MonoSpark simply by changing herbuild file to refer to MonoSpark rather than Spark. MonoSparkinherits most of the Spark code base, and the application coderunning on Spark and MonoSpark is identical. For example,for a job that filters a dataset from disk and saves the result,Spark and Monotasks both read the same input data from thesame place on disk, use the same code to deserialize the inputdata, run exactly the same Scala code to perform the filter,use the same code to serialize the output, and write identicaloutput to disk. MonoSpark only changes the code that handlespipelining resources used by a task.

HDFS integration As part of integrating with Spark, ourimplementation also integrates with Hadoop Distributed FileSystem (HDFS) [6], which is commonly used to store datathat is analyzed with Spark. Spark uses a pipelined version of

2The MonoSpark code and scripts to run the experiments in the evaluationare available at https://github.com/NetSys/spark-monotasks.

the HDFS API, where Spark reads one deserialized record ata time, and HDFS handles pipelining reading data from diskwith decompressing and deserializing the data (writes work ina similar manner). In order to separate the disk and computemonotasks, we re-wrote the HDFS integration to decoupledisk accesses from (de)serialization and (de)compression. Wedid this using existing HDFS APIs, so we did not need tomodify HDFS.

Resource schedulers All of the per-resource schedulersare written at the application level and not within the op-erating system, meaning that resource use is not perfectlycontrolled. For example, we run one CPU monotask for eachCPU core, but we do not pin these tasks to each core, andtasks may be interrupted by other monotasks that need smallamounts of CPU (e.g., to initiate a disk read). We find thatMonoSpark enables reasoning about performance in spite ofthese imperfections.

5 MONOTASKS PERFORMANCEIn this section, we compare performance of MonoSpark toSpark. We find that changing fine-grained pipelining to coarse-grained, single-resource monotasks does not sacrifice perfor-mance: MonoSpark provides job completion times within 9%of Apache Spark for typical scenarios.

5.1 Experimental setupWe ran all of our experiments on clusters of Amazon EC2instances that have 8 vCPUs, approximately 60GB of mem-ory, and two disks. Some experiments use m2.4xlarge in-stances with two hard disk drives (HDDs), while others usesi2.2xlarge instances with one or two solid state drives (SSDs),in order to illustrate that monotasks works with both typesof disks. Our experiments compare Spark version 1.3 andMonoSpark, which is based on Spark 1.3. We ran at leastthree trials of each experiment, in addition to a warmup trial(to warmup the JVM) that was discarded, and except whereotherwise noted, all plots show the median with error bars forthe minimum and maximum values.

The version of Spark we compared against is known tohave various CPU inefficiencies, and recent efforts have pro-duced significant improvements in runtime, both in newerversions of Spark [33], and in alternative systems [10, 21]. Asdescribed in §4, we changed only the parts of Spark relatedto resource pipelining, so MonoSpark inherits Spark’s CPUinefficiencies. MonoSpark would similarly inherit recent op-timizations, which are orthogonal to the use of monotasks.For example, efforts to reduce serialization time would re-duce the runtime for the compute monotasks that perform(de)serialization in MonoSpark. We also designed our work-loads to avoid any single bottleneck resource, as we elaborate

Page 8: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

���

���

���

���

���

���

���

�� �� �� �� �� �� �� �� �� �

����������

�����

��������������

������ ����� �� ����

Figure 5: Comparison of Spark and MonoSpark forqueries in the big data benchmark, using scale factor of 5,compressed sequence files, and 5 worker machines. Twoconfigurations of Spark are shown: the default, and aconfiguration where Spark writes through to disk ratherthan leaving disk writes in the buffer cache.

on in §5.2, and to include a workload (the machine learningworkload) that avoids Spark-specific CPU overheads.

5.2 Does getting rid of fine-grained pipelininghurt performance?

We evaluate MonoSpark using three workloads that representa variety of applications and a variety of performance bot-tlenecks. In order to effectively evaluate whether changingfine-grained pipelining to single-resource monotasks harmsperformance, we chose workloads that do not have one re-source with dramatically higher utilization than all other re-sources, as we elaborate on below. Workloads with severalhighly utilized resources put the most stress on strategies thatre-arrange resource use, because resource use needs to beparallelized to maintain fast job completion time.

This section presents the high-level results of comparingMonoSpark to Spark for the three benchmark workloads; §5.3and §5.4 describe differences in performance in more detail.

Sort The first workload sorts 600GB of random key-valuepairs from disk using twenty worker machines that each havetwo hard disk drives. The job reads the input data from HDFS,sorts it based on the key, and stores the result back in HDFS.We tuned the workload to use CPU and disk roughly equallyby adjusting the size of the value (smaller values result inmore CPU time, as we elaborate on in §6.2). For this work-load, Spark sorts the data in a total of 88 minutes (36 minutesfor the map stage and 52 minutes for the reduce stage), andMonoSpark sorts the data in 57 minutes (22 minutes for themap stage and 35 minutes for the reduce stage). The reasonMonoSpark is faster than Spark for this workload is discussedfurther in §5.4.

��������������

���� ����������������

��� ���� ����������������

�����������

��������������

Figure 6: Utilization of the most utilized (i.e., bottleneck)resource, and the second most utilized resource duringstages in the big data benchmark, for both Spark andMonoSpark. Boxes show the 25, 50, and 75th percentiles;whiskers show 5th and 95th percentiles.

Big Data Benchmark The big data benchmark [31] wasdeveloped to evaluate the differences between analytics frame-works and was derived from a benchmark developed by Pavloet al. [27]. The input dataset consists of HTML documentsfrom the Common Crawl document corpus [2] combinedwith SQL summary tables generated using Intel’s Hadoopbenchmark tool [35]. The benchmark consists of four queriesincluding two exploratory SQL queries, one join query, andone page-rank-like query. The first three queries have threevariants that each use the same input data size but have dif-ferent result sizes to reflect a spectrum between business-intelligence-like queries (with result sizes that could fit inmemory on a business intelligence tool) and ETL-like querieswith large result sets that require many machines to store. Thefourth query performs a transformation using a Python script.We use the same configuration that was used in publishedresults [31]: we use a scale factor of five (which is the largestscale factor available) and a cluster of five worker machinesthat each have two HDDs.

Figures 5 compares runtime with MonoSpark to runtimewith Spark for each query in the workload. For all queriesexcept 1c, MonoSpark is at most 5% slower and as muchas 21% faster than Spark. Query 1c takes 55% longer withMonoSpark, which we discuss in more detail, along with thesecond configuration of Spark, in §5.3.

Figure 6 shows the resource utilization of the two mostutilized resources on each executor during each stage of thebig data benchmark queries, and illustrates two things. First,multiple resources were well-utilized during most stages. Sec-ond, MonoSpark utilized resources as well as or better thanSpark.

We also ran the big data benchmark queries on machineswith two SSDs (as opposed to two HDDs), to understand howdifferent hardware affects the relative performance of Sparkand MonoSpark. On SSDs, the MonoSpark is at most 1%slower than Spark and up to 24% faster; detailed results areomitted for brevity.

Page 9: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

���������������

������������ �

���� ������ ������������ �

�����

��������������������

��������������

Figure 7: Comparison of Spark and MonoSpark for eachstage of a machine learning workload that computes aleast squares fit using 15 machines.

Machine Learning The final workload is a machine learn-ing workload that computes a least squares solution using aseries of matrix multiplications3 on a cluster of 15 machines,each with 2 SSDs. Each multiplication involves a matrix withone million rows and 4096 columns. Tasks in the workloadeach perform a matrix multiplication on a block of rows. Thisworkload differs from the earlier workloads for three reasons.First, it has been optimized to use the CPU efficiently: matri-ces are represented as arrays of doubles that can be serializedquickly, and each task calls out of the JVM to optimized nativecode written using OpenBLAS [3]. Second, a large amountof data is sent over the network in between each stage; com-bined with the fact that CPU use has been optimized, thisworkload is network-intensive. Finally, the workload does notuse disk, and stores shuffle data in-memory. As with otherworkloads, MonoSpark provides performance on-par withSpark, as shown in Figure 7.

5.3 When is MonoSpark slower than Spark?MonoSpark can be slower than Spark in two cases. First,MonoSpark can be slower when a workload is not broken intosufficiently many multitasks. By using monotasks, MonoSparkeliminates fine-grained pipelining, a technique often consid-ered central to performance. MonoSpark’s coarser-grainedpipelining will sacrifice performance when the pipelining istoo coarse, which occurs when the job has too few multitasksto pipeline one multitask’s disk read, for example, with anearlier multitask’s computation. Figure 8 plots the runtime ofMonoSpark and Spark for a workload that reads input datafrom disk and then performs a computation over it, usingdifferent numbers of tasks. For each number of tasks, werepartition the input data into a number of partitions equalto the desired number of tasks. When the number of tasks isequal to the number of cores (the left most point), MonoSparkis slower than Spark, but as the number of tasks increases,

3The workload uses the block coordinate descent implementation in a dis-tributed matrix library for Apache Spark, available at https://github.com/amplab/ml-matrix/.

��

��

��

��

��� ��� ��� ���� ����

��������������������

������ �� �����

��������������

Figure 8: Comparison of runtime with Spark andMonoSpark for a job that reads input data and then com-putes on it, running on 20 workers (160 cores). Spark isfaster than MonoSpark with only one or two waves oftasks, but by three waves, MonoSpark’s pipelining acrosstasks has overcome the performance penalty of eliminat-ing fine grained pipelining.

MonoSpark can do as well as Spark by pipelining at the gran-ularity of monotasks. This effect did not appear in any of thebenchmark workloads that we ran because the default con-figuration for all of those workloads broke jobs into enoughtasks to enable pipelining across different monotasks.

The second reason MonoSpark may be slower than Sparkstems from how disk writes are treated: Spark writes data tobuffer cache, and does not force data to disk. Disk monotasks,on the other hand, flush all writes to disk, to ensure thatfuture disk monotasks get dedicated use of the disk, andbecause the ability to measure the disk write time is criticalto performance clarity. This difference explains why query 1cin the big data benchmark was 55% slower with MonoSpark:each disk writes 511MB of data for MonoSpark but less than200KB for Spark. We configured the operating system toforce Spark to flush writes to disk and the resulting big databenchmark query runtimes are show in Figure 5. When Sparkwrites the same amount of output to disk as MonoSpark, query1c is only 9% slower with MonoSpark.

5.4 When is MonoSpark faster than Spark?In some cases, MonoSpark can provide faster runtimes thanSpark. This is occurs for two reasons. First, per-resourceschedulers control contention, which results in higher diskbandwidth for workloads that run on hard disk drives, due toavoiding unnecessary seeks. The effect explains MonoSpark’sbetter performance on the sort workload and on some queriesin the big data benchmark, where controlling disk contentionresulted in roughly twice the disk throughput compared towhen the queries were run using Spark.

Second, the per-resource schedulers allow monotasks tofully utilize the bottleneck resource without unnecessary

Page 10: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

��������������������

���� ���� ���� ���� ���� ���� ����

�����������

��������

�����

��� ������ ������

��������������������

���� ���� ���� ���� ���� ���� ����

�����������

��������

���������

Figure 9: Utilization during the map stage of query 2c inthe big data benchmark. With MonoSpark, per-resourceschedulers keep the bottleneck resource fully utilized.

contention. Figure 9 shows one example when MonoSparkachieved better utilization. The figure plots the utilization onone machine during a thirty second period in the map stage ofquery 2c in the big data benchmark. Over the entire map stage,the per-resource schedulers in MonoSpark keep the bottleneckresource, CPU, fully utilized: the average utilization is over92% for all machines. With Spark, on the other hand, eachtask independently determines when to use resources. As aresult, at some points, tasks bottleneck on the disk while CPUcores are unused, leading to lower utilization of the CPU (75- 83% across all machines) and, as a result, longer completiontime. This problem would persist even with proposals thatuse a task resource profile to assign tasks to machines [16].Resource profiles based on average task resource use do notaccount for how resource consumption arrives over the courseof the task, so cannot avoid contention caused by fine-grainedchanges in resource use.

6 REASONING ABOUT PERFORMANCEExplicitly separating the use of different resources into mono-tasks allows each job to report the time spent using each re-source. These times can be used to construct a simple modelfor the job’s completion time, which can be used to answerwhat-if questions about completion time under different hard-ware or software configurations. In this section, our goal is notto design a model that perfectly captures job completion time;instead, our goal is to design a model that is simple and easy touse, yet sufficiently accurate to provide estimates to users whoare evaluating the benefit of different hardware or softwareconfigurations. We use the model to answer four “what-if”questions about how a job’s runtime would change using adifferent hardware configuration, software configuration, or acombination of both. The model provides predictions within28% of the actual runtime – even when the runtime changesby as much as a factor of 10. In addition, we show that our

Ideal CPU time: total CPU monotask time /

# CPU cores

Ideal network runtime

Ideal disk runtime(bottleneck)

Job runtime: max of ideal times

Ideal CPU time(bottleneck)

Ideal network runtime

Ideal disk runtime with 2x disk throughput

Job runtime with 2x disk throughput

Figure 10: Monotask runtimes can be used to model jobcompletion time as the maximum runtime on each re-source. This example has 4 CPU cores and 2 hard disks.

model makes it trivial to determine a job’s bottleneck anddemonstrate its use to replicate the bottleneck analysis resultsfrom [25]. Finally, we evaluate the effectiveness of applyingthe same model to Spark.

6.1 Modeling performanceDecomposing jobs into monotasks leads to a simple modelfor job completion time. To model job completion time, eachstage is modeled separately (since stages may have differ-ent bottlenecks), and the job completion time is the sumof the stages’ completion times. Modeling the completiontime of a stage involves two steps. First, information aboutthe monotasks can be used to compute the ideal time spentrunning on each resource, which we call the ideal resourcecompletion time. For the CPU, the ideal time is the sumof the time for all of the compute monotasks, divided bythe number of cores in the cluster. For example, if a job’sCPU monotasks took a total of twenty minutes, and the jobran on eighty cores, the ideal CPU time would be fifteenseconds, because all of the CPU monotasks could have com-pleted in fifteen seconds if perfectly parallelized. For I/Oresources, the ideal resource time can be calculated usingthe total data transmitted and the throughput of the resource:ideal I/O resource time = sum of data transferred

resource throughput . For example, ifa stage read twenty gigabytes from disk, and used ten disksthat each provided 100 megabytes per second of read through-put, the ideal disk time would be twenty seconds.

The second step in building the model is to compute theideal stage completion time, which is simply the maximumof any resource’s completion time, as shown in Figure 10. Inessence, the ideal stage completion time is the time spent onthe bottleneck resource.

This model is simple and ignores many practicalities, in-cluding the fact that resource use cannot always be perfectlyparallelized. For example, if one disk monotask reads muchmore data than the other disk monotasks, the disk that exe-cutes that monotask may be disproportionately highly loaded.Furthermore, all jobs have a ramp up period where only oneresource is in use; e.g., while the first network monotasks are

Page 11: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

��������������������������������������

��������� ��������� ���������

�����������

��������

�������������������������������������

������������������

Figure 11: Monotask runtimes can be used to model jobruntime on different cluster configurations. In this exam-ple, monotask runtimes from experiments on a cluster of20 8-core, 1 SSD machines was used to predict how muchfaster the job would run on a cluster with twice as manydisks on each machine.

executing for a reduce task. Despite these omissions, we findthat our model is sufficiently accurate to answer broad what-ifquestions, which we believe is preferable to a perfectly ac-curate but complex model that is difficult to understand andapply.

The model for job completion time can trivially be used todetermine the bottleneck, which is simply the resource withthe longest ideal resource completion time.

6.2 Predicting runtime on different hardwareWhile understanding the bottleneck is useful, the model pro-vides the most value in allowing users to answer what-ifquestions about performance. For example, to compute howmuch more quickly a job would complete if the job had twiceas much disk throughput available, the ideal disk time wouldbe divided by two, and the new modeled job completion timewould be the new maximum time for any of the resources.An example of this process is shown on the right side of Fig-ure 10. To compute the new estimated job completion time,we scale the job’s original completion time by the predictedchange in job completion time based on the model. This helpsto correct for inaccuracies in the model; e.g., not modelingtime when resource use cannot be perfectly parallelized.

Figure 11 illustrates the effectiveness of the model in pre-dicting the runtime on a cluster with twice as many SSDsfor three different versions of a sort workload. The figureshows the runtime on a 20-machine cluster with one SSD perworker, the predicted time on a cluster with two SSDs perworker (based on monotask runtimes on the one SSD clus-ter), and the actual runtimes on a cluster with two SSDs perworker. The workload sorts 600GB of key-value pairs, whereeach value is an array of longs. Increasing the size of thevalue array while fixing the total data size decreases the CPUtime (because fewer keys need to be sorted) while keeping

��������������������������������

�� �� �� �� �� �� �� �� �� �

����������

�����

�������� ������� ���������������� ������

������ ������

Figure 12: For all queries in the big data benchmarkexcept 3c, the monotasks model correctly predicts thechange from changing each machine to have only 1 diskinstead of 2.

the I/O demand fixed. We change the size of the value array(to be 10, 20, and 50) to evaluate how effective the modelis for different balances of resources. With only 10 valuesassociated with each key, the workload is CPU-bound, so themodel predicts no change in the job’s completion time as aresult of adding another disk on each worker. In this case,the error is the largest (9%), because the workload does see amodest improvement from adding an extra disk (this improve-ment stems from reducing the duration of transient periodswhere the disk is the bottleneck, e.g., when starting a stage,where data must be read from disk before other resources canstart). For the other two workloads, the model predicts the cor-rect runtime within a 5% error. For both of these workloads,computing the new runtime with twice as many disks isn’tas simple as dividing the old runtime by two. In at least oneof the stages of both workloads, adding an extra disk shiftsthe bottleneck to a different resource (e.g., to the network)leading to a smaller than 2⇥ reduction in job completion time.The monotasks model correctly captures this.

Figure 12 uses the model to predict the effect of removingone of the two disks on each machine in the big data bench-mark workload from §5. A user running the workload mightwonder if she could use just one disk per machine instead oftwo, because most queries in the workload are CPU bound(as discussed further in §6.5). The monotasks model correctlypredicts that most queries change little from eliminating adisk: the predictions for all queries except query 3c are within9% of the actual runtime.

The prediction has highest error for query 3c, when itoverestimates the new runtime by 28%. This occurs becausemonotasks incorrectly predicts the change in runtime for alarge shuffle stage that makes up approximately half of thejob runtime. On-disk shuffles are the most difficult types ofstage in which to get high utilization for both MonoSpark andSpark, because maintaining high utilization for the multitasksrunning on any one machine requires a steady flow of shuffle

Page 12: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

data being read from all of the other machines in the cluster.A single slow disk can block progress on all other machineswhile those machines wait for shuffle data from the slow disk.This problem is most pronounced when all three resources areused equally, in which case all resources on all machines mustconsistently be driven to high utilization to get the best perfor-mance. All three resources are used equally for the two-diskconfiguration, and performance is poor as a result: the averageCPU, disk, and network utilization on each executor duringthis stage is roughly 50%. Since all resources are evenly bot-tlenecked, the model predicts that the runtime for that stagewill increase by a factor of two once a disk is removed. In-stead, when a disk is removed, MonoSpark is able to drivethe bottleneck resource (now, the disk) to a higher utilization,and as a result, the stage only gets approximately 40% slower.Improving monotasks scheduling to enable uniformly higherresource utilizations would reduce this modeling error.

6.3 Predicting runtime with deserialized dataThe model can also be used for more sophisticated what-ifscenarios; e.g., to estimate the improvement in runtime if in-put data were stored in-memory and deserialized, rather thanserialized on disk. This requires modeling two changes. First,data no longer will be read from disk. We account for thischange by not including time for disk monotasks that readinput data when we compute the ideal disk time (this is pos-sible because each monotask reports metadata that includeswhether the task was to read input or write output). Second,the job will spend less time using the CPU, because inputdata is already deserialized. MonoSpark separates the com-pute monotask into a first part that deserializes all of the data,and a second part that performs the remaining computation,and the compute monotask reports how long each part took.To model job completion time when data is already deseri-alized, we do not include the time to deserialize input datawhen modeling ideal CPU time. Using the new modeled idealtimes, we calculate the new job completion time. We used thisapproach to predict the runtime of a job that sorted random on-disk data if data were stored deserialized in-memory, and themodel predicted the new runtime within an error of 4% (themodel predicted that the job’s runtime would reduce from48.5 seconds to 38.0 seconds, and the job’s actual runtimewith in-memory data was 36.7 seconds).

Separating the deserialization time from the rest of thecomputation is only possible because of the use of monotasks.Deserialization time cannot be measured in Spark because ofrecord-level pipelining: Spark deserializes a single record andcomputes on that record before deserializing the next record,and processing a single record is too fast to time withoutsignificant overhead.

��������������������������������������

��������� ��������� ���������

�����������

��������

��������������������������������������������������������������������������������������������������

�������������������������������������������������

Figure 13: The monotasks model predicts the 10⇥ im-provement in runtime resulting from moving a workloadfrom 5 machines with hard-disk-drives and input storedon-disk to 20 machines with solid-state-drives and inputstored deserialized and in-memory with an error of atmost 23%.

6.4 Predicting with both hardware andsoftware changes

Thus far, we have shown how the monotasks model can beused to predict runtime changes resulting from a single changeto the hardware or software configuration. The simple modelcan also accurately predict runtime changes stemming frommultiple changes to both hardware and software. Figure 13 il-lustrates the change in runtime for 3 workloads that are movedfrom a 5-machine HDD cluster to a 20-machine SSD cluster.All workloads read 100GB of input data, sort it, and write itback to disk; the three workloads vary in the ratio of CPU todisk time, as in §6.2. The 100GB of input data did not fit inmemory on the 5 machine cluster (it takes up approximately200GB in memory), but with 20 machines, there is enoughcluster memory to store input data in a deserialized format.In total, there are three changes associated with moving theworkload to 20 machines: the workload runs on 4⇥ as manymachines (the number of tasks stays constant), input data isstored deserialized and in-memory rather than on-disk, andmachines each have 2 SSDs rather than 2 HDDs, so shuffleand output data can be read and written more quickly. Themonotasks model correctly predicts the resulting 10⇥ changein runtime with an error of 23% in the worst case.

To give an example of the changes that occurred as a resultof the hardware and software changes, the workload with 10values was bottlenecked by disk on the 5-machine clusterin both the map and reduce stages. The monotasks modelcorrectly predicted that, given the hardware changes, the mapstage became CPU-bound (due to a combination of inputdata being stored in-memory, and the faster write time for theshuffle data), and the reduce stage became network bound(due to the faster read time for shuffle data and write timefor output). One source of error for all three workloads was

Page 13: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

���

���

���

���

���

���

�� �� �� �� �� �� �� �� �� �

����������

�����

�������� ��������� ����������� ������� ���� �� ������� ������� ���� �� ����

���� ������� ���� �� �������

Figure 14: Monotask runtimes can be used to replicateprevious work that used extensive logging to determinethe job completion time if different resources were opti-mized to be infinitely fast (which serves as a bound onimprovements from optimizing that resource).

the network time. The model assumed that the same amountof data was sent over the network in both the 5-machine and20-machine case. However, with 5 machines, each task couldread 20% of its input data locally, whereas with 20-machines,an average of only 5% of input data could be read locally. Asa result, the workloads on 20-machines sent more data overthe network (and, correspondingly, spent more time using thenetwork) than the model predicted.

6.5 Understanding bottlenecksIn addition to predicting runtime under different hardware andsoftware configurations, our model can be used for simplerbottleneck analysis. Recent work [25] dedicated to answeringthat question added instrumentation to Spark to measure timesblocked on the network and disk, and used those measure-ments to determine the best-case improvement from optimiz-ing disk and network: i.e., it used blocked times to determinehow much faster jobs would run if they did not spend anytime blocked on a particular resource. The analysis relied onadding extensive white-box logging to Spark; with monotasks,the necessary instrumentation (i.e., the runtime of differenttypes of monotasks) is built into the framework’s executionmodel. Typically the monotasks model predicts job comple-tion time by taking the maximum of the ideal CPU, network,and disk times. The best-case job completion time if the diskwere optimized can be computed by simply excluding thedisk from the maximum, so instead taking the maximum overthe ideal network and CPU times. Figure 14 illustrates thesepredictions using monotask runtimes for the big data bench-mark used in [25]. We arrive at the same findings as [25]:for the big data benchmark, CPU is the bottleneck for mostqueries, improving disk speed could reduce runtime of somequeries, and improving network speed has little effect. Somequeries see improvement from optimizing multiple resources

��������������������������������

�� �� �� �� �� �� �� �� �� �

����������

�����

�������� ���������������� ������ ����� �� ������

������ ������

Figure 15: Using the number of slots to model the changein job completion time from reducing the number ofdisks does not result in correct predictions.

(e.g., query 3c) because the query consists of multiple stagesthat each have different bottlenecks.

6.6 Can this model be used for Spark?This section explores whether the model we used to predictperformance with MonoSpark could be applied to Spark,without re-architecting internals. Creating a model for jobcompletion time is easy in MonoSpark because instrumenta-tion for use of different resources is built into the frameworkarchitecture: resource use is separated into monotasks, andeach monotask reports how long it took to complete. WhileMonoSpark controls resource use with per-resource mono-tasks, Spark controls resource use with slots: as mentionedin §3.4, the job scheduler assigns tasks to machines in a fixednumber of slots, and controlling this number of slots is theonly mechanism the scheduler has for regulating resource use.The most straightforward way to apply the monotasks modelto Spark is to use slots in the same way that the monotasksmodel uses monotasks: for example, if a job took 10 secondsto complete on a cluster with 8 slots, it should take 5 secondsto complete on a cluster with 16 slots. Figure 15 illustratesthe effectiveness of this model, for the same prediction shownin Figure 12, where the runtime of the big data benchmark onmachines with 2 HDDs is used to predict runtime on a clusterwith 1 HDD per machine. Spark sets the number of slots to beequal to the number of CPU cores, so changing the number ofdisk drives does not change the number of slots. As a result,this model is inaccurate: it does not account for the slowdownthat occurs when queries become disk bound. The user couldscale the number of slots to 4 rather than 8, to account for thereduction in disk usage, but this would lead to predicting thatqueries would take twice as long with the reduced number ofdisks, which is only true for disk-bound queries. Fundamen-tally, the problem is that Spark uses one dimension, slots, tocontrol resource use that is multi-dimensional.

Unfortunately, measuring Spark’s resource usage to cre-ate a more sophisticated model is difficult. Spark tasks on

Page 14: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

���

���

���

���

���

���

���

����������

����� ����� �����

���

����

����

����

����

����

����

����

������

���������

���

���

���

���

���

���

��������������

���������

������

Figure 16: When multiple jobs are run concurrently inSpark, attributing resource to a particular job is difficultto do accurately. For one stage of one of the jobs, ourestimate of resource usage was consistently inaccurate.

a machine all run in a single process (the Java virtual ma-chine), and resource usage of different concurrent tasks isinterleaved by both Spark and the operating system sched-uler. We measured this effect by running two different sortworkloads concurrently: the 10-value and 50-value workloadsfrom §6.2. We estimated resource use for each job by mea-suring the total resource use on each executor while eachstage was running, and estimating each stage’s resource useby scaling each executor’s total resource use by the fractionof slots that were used by tasks for the stage. For example,if a stage ran for 10 seconds, and one executor with eightslots ran four, five-second tasks, we would divide the totalresource use on the executor by four. Figure 16 shows this es-timate, for the map stage in the 10-value job. These estimatesare consistently incorrect, sometimes by a factor of two ormore, because resource use is attributed equally to both jobs,rather than accounting for the jobs’ different resource pro-files. The median and 75th percentile error for all resources inboth stages of both jobs is 17% and 68%, respectively, withSpark. Monotask times can easily be used to decouple re-source use for the same two jobs: with MonoSpark, the erroris consistently less than 1%.

Figure 16 illustrated why measuring resource use in Sparkis difficult. If we could measure the resource use in Spark,would the model be accurate? While we cannot measure theresource use of each task, the monotask model relies on takingthe aggregate resource use for an entire stage (by summing allof the monotask times for each resource, in each stage). Weapproximated this process in Spark by measuring the resourceuse on each executor while the big data benchmark is runningin isolation. Because no other workloads were running, all ofthe resource use on each executor can be attributed to the stage.We used these resource uses as inputs to the monotasks model;Figure 17 illustrates the results. The model underestimatesthe increase in job completion time that results from usingonly one disk: the error ranges from 3%, for query 1a, to over50%, for query 1c. This error stems from the challenges toreasoning about performance outlined in §2. In particular, one

��������������������������������

�� �� �� �� �� �� �� �� �� �

����������

�����

�������� ���������������� ������ ����� �� �������� ����

������ ������

Figure 17: Even in cases where Spark’s resource use canbe measured to make a more accurate model than theslot-based approach in Figure 15, a Spark-based modelhas an error of 20-30% for most queries.

source of error is contention: when all tasks use one disk,contention leads to reduced disk throughput, which is notcaptured by the model.

We are able to model Spark performance only in a restrictedcase (when a job runs in isolation) and even in this case,the error was higher than the error for the same scenariousing MonoSpark. This model can only be used for hardwarechanges; as mentioned in §6.3, it is not possible to measuredeserialization time in Spark.

7 LEVERAGING CLARITY:AUTO-CONFIGURATION

Because MonoSpark explicitly schedules the use of each re-source, the framework has better visibility to automaticallyperform configuration that users are typically required to do.One configuration parameter that MonoSpark can set auto-matically is the appropriate amount of concurrency on eachworker. Spark sets the number of concurrent tasks on eachworker to be the number of cores on the worker, and allowsusers to change this by setting a configuration parameter. Dif-ferent configurations will work better for different workloads;for example, the user might want to increase concurrencyto more than the number of cores if each worker machinehas a large number of disks, and tasks are not CPU-intensive.MonoSpark eliminates this configuration parameter, becauseconcurrency is controlled by each resource scheduler, as de-scribed in §3.4.

Figure 18 compares performance with MonoSpark to per-formance under a variety of Spark configurations, for threedifferent workloads. The workloads each sort randomly gen-erated key-value pairs by key, and differ in the number ofvalues associated with each key, as in the workload in §6.2.The different characteristics of each workload mean that thebest Spark configuration differs across workloads, and to ex-tract optimal performance, a user would need to tune this

Page 15: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

��

���

���

���

���

����

����

������ ������ ������ ������ ������� ����

�������������������������� ��������

���������������������

(a) Single Long

�����������������������������

������ ������ ������ ������ ����

�������������������������� ��������

���������������������

(b) 25 Longs

��������������������������

������ ������ ������ ������ ����

�������������������������� ��������

���������������������

(c) 100 Longs

Figure 18: Runtimes for three different jobs, each under different configured numbers of tasks per machine with Spark(e.g., Spark2 is Spark with 2 tasks per machine). MonoSpark automatically configures the number of tasks per machine,and performs at least as well as the best Spark configuration for all three jobs.configuration parameter to the correct value. MonoSpark au-tomatically uses the ideal amount of concurrency for eachresource, and as a result, performs at least as well as thebest Spark configuration for all workloads. In some cases,MonoSpark performs as much as 30% better than Spark. Thisis for two reasons: first, Spark doesn’t allow users to changethe number of concurrent tasks between the stages, and some-times the ideal value differs for the two stages. Second, thedisk monotask scheduler controls concurrent disk accesses toavoid unnecessary seeking.

8 LIMITATIONS AND OPPORTUNITIESThis section begins with fundamental limitations of usingmonotasks, and then discusses opportunities to improve ourcurrent implementation.

Jobs with few tasks As illustrated in §5.3, if a job has asmall number of multitasks relative to the available resources,then there may be no opportunities for coarse-grained pipelin-ing across different monotasks. For the benchmark workloadswe ran, we did not find this to be a problem, because thedefault configuration of all three workloads broke jobs intoa sufficiently large number of multitasks. Many frameworkstoday encourage the use of a large number of small multitasksfor performance reasons: having a large number of small tasksmitigates the effect of stragglers [24, 34] and helps to avoidsituations where the task’s intermediate data is so large that itneeds to be spilled to disk [29]. Workloads that have only asingle wave of multitasks will need to be broken into a largernumber of smaller multitasks in order to run efficiently withmonotasks, which can be done by changing a parameter whenjobs are submitted.

Jobs with large tasks Frameworks like Spark and Hadoopallow tasks to handle more data than can fit in memory: ifa task’s intermediate data does not fit in memory, tasks willspill the data to disk and merge it at the end. However, inMonoSpark, data must fit in a buffer in-memory after it isread from (or before it is written to) persistent storage. Asis the case for jobs with too few tasks, jobs with multitasks

that are too large will need to be run with a larger number ofsmaller multitasks with monotasks.

Head of line blocking A monotask that reads a largeamount of data from disk may block other tasks reading fromthat disk. This is not an issue with current frameworks be-cause tasks share access to each resource at fine granularity.Using smaller tasks mitigates this problem with monotasks.

Memory use As mentioned in §3.5, MonoSpark uses morememory than Spark, because data is materialized in-memorybetween different types of monotasks. This memory pres-sure can alleviated by breaking jobs into a larger number ofmultitasks, which results in each multitask operating on lessdata, and, as a result, less data needing to be kept in-memory.The monotasks design could also be augmented to includea memory manager similar to the memory manager used inThemis [28].

Caching data Disk monotasks do not write data to thebuffer cache because letting the OS manage the buffer cachehurts predictability. This hurts our performance compared toSpark for some jobs, as described in §5.3. MonoSpark couldleverage Spark’s application-level cache to opportunisticallyavoid writing data to disk.

Disk scheduling The disk monotask scheduler currentlybalances requests across available disks, independent of load.A better strategy would consider the load on each disk indeciding which disk should write data; for example, writingto the disk with the shorter queue.

Multitask scheduling Our current implementation usesa simple multitask scheduler that assigns up to a maximumnumber of multitasks to each machine. This scheduler couldbe used to implement more sophisticated policies, e.g., toshare machines between different users.

9 RELATED WORKPerformance clarity Most existing work approaches perfor-mance clarity by treating the system architecture as fixed, andeither adding instrumentation or performing black-box exper-iments to reason about performance [4, 7, 17, 25]. Section 2.2described existing approaches to performance clarity for data

Page 16: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

SOSP ’17, October 28, 2017, Shanghai, China K. Ousterhout et al.

analytics frameworks. Ongoing debate about the bottleneckfor data analytics frameworks suggests that reasoning aboutperformance remains non-trivial [22, 23, 25, 30].

Performance clarity has been studied more extensively inthe context of high-performance single-server applications.Causal profiling simulates the impact of performance opti-mizations by measuring the relative impact on performanceof slowing down concurrent code [11]. If applied to large-scale distributed systems, causal profiling could answer thetype of what-if questions that we used monotasks to answer.Flux takes an architectural approach, similar to monotasks:engineers use the Flux language to write high-performanceservers, which enables both bottleneck identification and per-formance prediction [8].

Improving analytics system performance Numerous re-cent efforts have improved the performance of data ana-lytics frameworks by optimizing CPU use; e.g., by reduc-ing serialization cost, structuring computation to more effi-ciently use CPU caches, taking advantage of vectorization,and more [10, 22, 23, 33]. These efforts are orthogonal tomonotasks: they make workloads less CPU-bound, but do notchange the fine-grained pipelining of today’s multi-resourcetasks.

Improving resource scheduling A variety of schedulingprojects have improved on slot-based scheduling models toaccount for use of multiple resources [14, 16]. These sched-ulers use estimates of task resource use to determine howmany tasks to assign to each machine. They treat the structureof the task as a black box, and do not schedule a task’s accessto each resource. As a result, tasks may still contend, even ifthe average resource use of each task is less than the amountof resources available on the machine, as discussed in §5.4.Improving resource throughput with per-resource schedulers,as is done with monotasks, has been explored in various sys-tems that use disk schedulers to batch access and avoid seeks(e.g., Themis [28] and Impala [9]).

Granularity of pipelining Traditional data processing sys-tems use fine-grained pipelining to stream records betweenoperators, as in the Volcano operator model [15]. Themis [28],for example, argues for record-at-a-time pipelining, whereeach record is processed fully after reading, to avoid memorypressure. We do not evaluate the very large scale (TBs ofdata) workloads targeted by Themis. As discussed in §8, forsuch workloads, we anticipate borrowing their insight for re-duced memory pressure: i.e., for such workloads, multi- andmonotasks should act on fewer records.

Fine-grained pipelining has been revisited in the databaseliterature to improve performance and take advantage of vec-torized execution [19, 26, 37]. For example, SQL server 2012abandoned the row-at-a-time iterator model, and instead pro-cesses a batch of (typically around 1000) rows at a time.Monotasks similarly abandons fine-grained pipelining, but

with the different objective of easing users ability to reasonabout performance bottlenecks.

10 CONCLUSIONThis paper explored a new system architecture designed toprovide performance clarity. We proposed decomposing to-day’s multi-resource tasks into smaller units of work, mono-tasks, that each use only one of CPU, network, or disk. Thisdecomposition trivially enables users to understand the sys-tem bottleneck, and also allows for accurate estimates ofthe impact of potential system changes. Using monotasksprovides performance clarity without sacrificing high perfor-mance. Performance with our implementation is comparableto Apache Spark, and using monotasks presents new opportu-nities for optimizations that we have not yet fully explored.

We believe that today, the primary obstacle to users whoare trying to improve performance of their workloads is notthat they have too few optimizations at their disposal, butrather that they do not know which optimization to choose.Furthermore, system bottlenecks are constantly changing, asdevelopers work to optimize current bottlenecks and as hard-ware changes. As a result, optimizations that were effectivea few months ago may no longer be useful. We hope that byillustrating that performance clarity can be provided as partof a system’s architecture, providing performance clarity –perhaps with monotasks – will be a first-class concern in thedesign of new systems.

ACKNOWLEDGMENTSWe indebted to Shivaram Venkataraman, for discussions dur-ing the tiny tasks project [24] that led to the idea of breakingjobs into small, single-resource units of work, and to MaxWolffe, for helping to implement disk optimization features inearly versions of MonoSpark. We thank Aurojit Panda, EddieKohler, and Patrick Wendell for providing helpful feedbackon earlier drafts of this paper. Finally, we thank our shepherd,Miguel Castro, for helping to shape the final version of thispaper. This research was supported in part by a Hertz Foun-dation Fellowship, a Google PhD Fellowship, and Intel andother sponsors of UC Berkeley’s NetSys Lab.

REFERENCES[1] Apache Spark: Lightning-Fast Cluster Computing. http://spark.apache.

org/.[2] Common Crawl. http://commoncrawl.org/.[3] OpenBLAS: An optimized BLAS library. http://www.openblas.net/.[4] M. K. Aguilera, J. C. Mogul, J. L. Wiener, P. Reynolds, and A. Muthi-

tacharoen. Performance Debugging for Distributed Systems of BlackBoxes. In Proc. SOSP, 2003.

[5] O. Alipourfard, H. H. Liu, J. Chen, S. Venkataraman, M. Yu, andM. Zhang. CherryPick: Adaptively Unearthing the Best Cloud Config-urations for Big Data Analytics. In Proc. NSDI, 2017.

Page 17: Monotasks: Architecting for Performance Clarity in Data ...kayousterhout.org/publications/sosp17-final183.pdf · Monotasks: Architecting for Performance Clarity in Data ... data,

Monotasks: Architecting for Performance Clarity SOSP ’17, October 28, 2017, Shanghai, China

[6] Apache Software Foundation. Apache Hadoop. http://hadoop.apache.org/.

[7] P. Barham, A. Donnelly, R. Isaacs, and R. Mortier. Using Magpie forrequest extraction and workload modelling. In Proc. SOSP, 2004.

[8] B. Burns, K. Grimaldi, A. Kostadinov, E. D. Berger, , and M. D. Corner.Flux: A Language for Programming High-Performance Servers. InProc. Usenix ATC, 2006.

[9] Cloudera. Cloudera Impala: Open Source, Interactive SQL for Hadoop.http://www.cloudera.com/content/cloudera/en/products-and-services/cdh/impala.html.

[10] A. Crotty, A. Galakatos, K. Dursun, T. Kraska, U. Çetintemel, and S. B.Zdonik. Tupleware: Redefining modern analytics. CoRR, 2014.

[11] C. Curtsinger and E. D. Berger. COZ: Finding Code that Counts withCausal Profiling. In Proc. SOSP, 2015.

[12] J. Dean and S. Ghemawat. MapReduce: Simplified Data Processing onLarge Clusters. In Proc. OSDI, 2004.

[13] P. Gao, A. Narayan, G. Kumar, R. Agarwal, S. Ratnasamy, andS. Shenker. pHost: Distributed Near-optimal Datacenter TransportOver Commodity Network Fabric. In Proc. CoNext, 2015.

[14] A. Ghodsi, M. Zaharia, B. Hindman, A. Konwinski, S. Shenker, andI. Stoica. Dominant Resource Fairness: Fair Allocation of MultipleResource Types. In Proc. NSDI, 2011.

[15] G. Graefe. Encapsulation of Parallelism in the Volcano Query Process-ing System. In Proc. SIGMOD, 1990.

[16] R. Grandl, G. Ananthanarayanan, S. Kandula, S. Rao, and A. Akella.Multi-Resource Packing for Cluster Schedulers. In Proc. SIGCOMM,2014.

[17] H. Herodotou. Hadoop performance models. CoRR, 2011.[18] M. Isard, M. Budiu, Y. Yu, A. Birrell, and D. Fetterly. Dryad: Dis-

tributed Data-Parallel Programs From Sequential Building Blocks. InProc. EuroSys, 2007.

[19] V. Leis, P. Boncz, A. Kemper, and T. Neumann. Morsel-Driven Paral-lelism: A NUMA-Aware Query Evaluation Framework for the Many-Core Age. In Proc. SIGMOD, 2014.

[20] N. McKeown. The iSLIP Scheduling Algorithm for Input-queuedSwitches. IEEE/ACM Trans. Netw., 7(2), 1999.

[21] F. McSherry, M. Isard, and D. G. Murray. Scalability! But at WhatCost? In Proc. Hot OS, 2015.

[22] F. McSherry and M. Schwarzkopf. The impact of fast networks ongraph analytics, part 1. http://tinyurl.com/qaw9lla.

[23] F. McSherry and M. Schwarzkopf. The impact of fast networks ongraph analytics, part 2. http://tinyurl.com/q7aeajb, 2015.

[24] K. Ousterhout, A. Panda, J. Rosen, S. Venkataraman, R. Xin, S. Rat-nasamy, S. Shenker, and I. Stoica. The Case for Tiny Tasks in ComputeClusters. In Proc. HotOS, 2013.

[25] K. Ousterhout, R. Rasti, S. Ratnasamy, S. Shenker, and B.-G. Chun.Making Sense of Performance in Data Analytics Frameworks. In Proc.NSDI, 2015.

[26] S. Padmanabhan, T. Malkemus, A. Jhingran, and R. Agarwal. Blockoriented processing of Relational Database operations in modern Com-puter Architectures. In Proc. ICDE, 2001.

[27] A. Pavlo, E. Paulson, A. Rasin, D. J. Abadi, D. J. DeWitt, S. Madden,and M. Stonebraker. A Comparison of Approaches to Large-scale DataAnalysis. In Proc. SIGMOD, 2009.

[28] A. Rasmussen, V. T. Lam, M. Conley, G. Porter, R. Kapoor, and A. Vah-dat. Themis: An I/O-efficient MapReduce. In Proc. SoCC, 2012.

[29] S. Ryza. How-to: Tune Your Apache Spark Jobs (Part 2). goo.gl/7gjmyfcontent_copyCopyshortURL, 2015.

[30] A. Trivedi, P. Stuedi, J. Pfefferle, R. Stoica, B. Metzler, I. Koltsidas,and N. Ioannou. On The [Ir]relevance of Network Performance forData Processing. In HotCloud, 2016.

[31] UC Berkeley AmpLab. Big Data Benchmark. https://amplab.cs.berkeley.edu/benchmark/, February 2014.

[32] S. Venkataraman, Z. Yang, M. Franklin, B. Recht, and I. Stoica. Ernest:Efficient Performance Prediction for Large-Scale Advanced Analytics.In NSDI, 2016.

[33] R. Xin and J. Rosen. Project Tungsten: Bringing SparkCloser to Bare Metal. https://www.databricks.com/blog/2015/04/28/project-tungsten-bringing-spark-closer-to-bare-metal.html, 2015.

[34] R. S. Xin, J. Rosen, M. Zaharia, M. J. Franklin, S. Shenker, and I. Stoica.Shark: SQL and Rich Analytics at Scale. In Proc. SIGMOD, 2013.

[35] L. Yi, K. Wei, S. Huang, and J. Dai. Hadoop Benchmark Suite (Hi-Bench). https://github.com/intel-hadoop/HiBench, 2012.

[36] M. Zaharia, M. Chowdhury, T. Das, A. Dave, J. Ma, M. McCauley,M. J. Franklin, S. Shenker, and I. Stoica. Resilient Distributed Datasets:A Fault-Tolerant Abstraction for In-Memory Cluster Computing. InProc. NSDI, 2012.

[37] P. Åke Larson, C. Clinciu, E. N. Hanson, A. Oks, S. L. Price, S. Ran-garajan, A. Surna, and Q. Zhou. SQL Server Column Store Indexes. InProc. SIGMOD, 2010.