CS6963 Distributed Systems

Lecture 13 Spark

Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing
Zaharia, Chowdhury, Das, Dave, Ma, McCauley, Franklin, Shenker, Stoica
NSDI 2012

  • Why use MapReduce?
  • Say looking through a log, why not implement it using the regular abstractions (sockets, files etc?)
    • Saves a lot of work:
      • communication between nodes
      • distribute code
      • schedule work
      • handle failures
  • The MapReduce paper had a lot of impact on big data analytics: simple and powerful.

    • But bit too rigid.
  • Other systems proposed fixes:

  • Dryad (Microsoft 2007)

    • Any directed acyclic graph, edges are communication channels, can be through disk or via TCP.
    • Can implement multiple iterations
    • Can pipeline through RAM, don't have to go to disk
    • Doesn't checkpoint/replicate, in the middle of the run (so failures can be expensive)
      • Run for hours, many iterations in, then lose part of prior result and have to re-run much work.
  • Pig latin (Yahoo 2008): programming language that compiles to MapReduce.

    • Adds "Database style" operators, mainly Join
    • Join: dataset 1 (k1,v1), dataset 2 (k1, v2). ==> (k1, v1, v2), takes cartesian product (all tuples of combinations of v1, v2 with same k1)
    • Example: dataset 1: all clicks on products on website, dataset 2: demographics (age of users), want average age of customer per product.
    • Allows multiple iterations
    • Can express more
    • Still has rigidness from MR (writes to disk after map, to replicated storage after reduce, RAM)

Spark

+-----+
| foo | HDFS
+-----+
   |
filter(v => v < 10)
   |
   V
+-----+
|     |
+-----+
   |
map(v => v + 1)
   |
   V
+-----+
|     |
+-----+
  • A framework for large scale distributed computation.
  • An expressive programming model
    • Iteration
    • Joins
  • User chains "transformations" on "RDDs" (essentially tables or relations).
    • Think "map", "reduce", "groupBy", "join".
    • Somewhere between MR and relational algebra.
    • hdfsFile('foo').filter(v => v < 10).map(v => v + 1)
    • When user tries to use the result it runs the transformations on the cluster to produce the output.
  • User provides hints to the scheduler about how to execute on cluster.
    • But, mostly the scheduler takes care of faults, moving data, etc.
  • A couple of big performance benefits here:
    • hdfsFile('foo').filter(v => v < 10).map(v => v + 1)
    • Fuses the transformations together to skip materializing them.
      • Result is another RDD
    • All of these functions are applied row-by-row.
    • No output is even produced until this RDD is passed to an action.
      • Action is something that actually demands the data in an RDD
      • e.g. print to screen, write to file, etc.
      • Causes the RDD to materialize.
    • No write back to HDFS and not even intermediates to memory.
  • Problem: what if we lose something part way through computation?
    • Rely on the key insight from MR! Determinism provides safe recompute.
    • Track 'lineage' of each RDD. Can recompute from parents if needed.
    • Interesting: only need to record tiny state to do recompute.
      • Need parent pointer, function applied, and a few other bits.
      • Log 10 KB per transform rather than re-output 1 TB -> 2 TB
  • Problem: what if we have a non-linear flow?
    • e.g. What if we want to use an RDD to two other RDDs?
    • Do we have have to recompute the parent RDD twice?
    • No, persist() lets the user materialize a result.
  • Lots more details:
    • How are things partitioned and scheduled across the cluster and more.
  • (Questions on high level?)

  • Relatively recent release, but used by (partial list) IBM, Groupon, Yahoo, Baidu..

  • Can get substantial performance gains when dataset (or a major part of it) can fit in memory, so anticipated to get more traction.

Logistic regression (from paper):
(Perhaps diagram what this is trying to do.)

val points = spark.textFile(...)
        .map(parsePoint).persist()
var w = // random initial vector
for (i <- 1 to ITERATIONS) {
    val gradient = points.map{ p =>
      p.x * (1/(1+exp(-p.y*(w dot p.x)))-1)*p.y
    }.reduce((a,b) => a+b)
    w -= gradient
}
  • w is sent with the closure to the nodes
  • How many RDDs are materialized here? A bit hard to see.
  • Looks like points plus gradient at each iteration.
    • Why? w is a shell local variable that has to be effected by the gradient at each step. Effecting the local variable is going to require a materialized result from the RDD so scala can do the actual subtraction on w at each iteration.
  • (Why persist()?)

PageRank (from paper):

val links = spark.textFile(...).map(...).persist() // (URL, outlinks)
var ranks = // RDD of (URL, rank) pairs
for (i <- 1 to ITERATIONS) {
  // Build an RDD of (targetURL, float) pairs
  // with the contributions sent by each page
  val contribs = links.join(ranks).flatMap {
     (url, (outlinks, rank)) =>
        outlinks.map(dest => (dest, rank/outlinks.size))
}
// Sum contributions by URL and get new ranks
  ranks = contribs.reduceByKey((x,y) => x+y)
     .mapValues(sum => a/N + (1-a)*sum)
}
  • Execution:

    • links: (URL, outlinks)
    • ranks: (URL, rank)
    • links.join(ranks): (URL, (outlinks, rank))
    • contribs: (destURL, [rank contrib slice])
    • ranksNext: sum ranks per destURL and bias result as per PR
  • Lineage graph (Figure 3)

    • Why persist on links? Used per iter, else reload from textFile each iter.
    • Why no persist on ranks? Original ranks is really only used once.
  • What is an RDD (table 3, S4)

    • function to compute
    • list of (parent RDD, wide/narrow dependency)
    • list of partitions
      • and whether/where they are materialized
    • partitioning scheme
      • Maps each record to a partition
    • computation placement hint
  • Each transformation takes (one or more) RDDs, and outputs the transformed RDD.

  • Q: Why does an RDD carry its partitioning info?

  • A: so transformations that depend on multiple RDDs know whether they need to shuffle data (wide dependency) or not (narrow)

    • Allows users control over locality and reduces shuffles.
  • Example:

Consider joins on:

+---+---+ +---+---+
|A-M|N-Z| |A-M|N-Z|
+---+---+ +---+---+

+-----------+-----------+ +-----------+-----------+-----------+
| h(k)%2==0 | h(k)%2==1 | | h(k)%3==0 | h(k)%3==1 | h(k)%3==2 |
+-----------+-----------+ +-----------+-----------+-----------+
  • (Draw in one more level above.)
  • Top case: need to release that result only depends on one partition in each parent; can avoid the need to "rebucket" the results
  • Bottom case:

    • If same number of partitions in hash partition, then wouldn't need to shuffle.
  • Q: Why the distinction between narrow and wide dependencies?

  • A: In case of failure.

    • Narrow dependency only depends on a few partitions that need to be recomputed.
    • Wide dependency might require an entire RDD
  • Handling faults.

    • When Spark computes, by default it only generates one copy of the result, doesn't replicate. Without replication, no matter if it's put in RAM or disk, if node fails, on permanent failure, data is gone.
    • When some partition is lost and needs to be recomputed, the scheduler needs to find a way to recompute it.
    • (faults can be detected by using heartbeat)
    • will need to compute all partitions it depends on, until a partition in RAM/disk, or in replicated storage.
    • if wide dependency, will need all partitions of that dependency to recompute, if narrow just one that RDD
  • So two mechanisms enable recovery from faults: lineage, and policy of what partitions to persist (either to one node or replicated) We talked about lineage before (Transformations)

  • The user can call persist on an RDD.

    • Materializes result in RAM if possible, spill to disk otherwise
    • With RELIABLE flag, will keep multiple copies
      • in RAM if possible, disk if RAM is full
    • Other options to replicate and to push to other media (e.g. HDFS, disks)
  • Q: Is persist a transformation or an action?

  • A: neither. It doesn't create a new RDD, and doesn't cause materialization. It's an instruction to the scheduler.

  • Q: By calling persist without flags, is it guaranteed that in case of fault that RDD wouldn't have to be recomputed?

  • A: No. There is no replication, so a node holding a partition could fail. Replication (either in RAM or in stable storage) is necessary

(Potentially skip this or go quick.) - Currently only manual checkpointing via calls to persist. - What is checkpointing? - Materialize all needed RDDs at some point in the computation to avoid wasted future work. - Basically what MR does at the end of every run. - Q: Why implement checkpointing? (it's expensive) - A: Long lineage could cause large recovery time. Or when there are wide dependencies a single failure might require many partition re-computations.

  • Checkpointing is like buying insurance: pay writing to stable storage so can recover faster in case of fault.

    • Depends on frequency of failure and on cost of slower recovery
    • An automatic checkpointing will take these into account, together with size of data (how much time it takes to write), and computation time.
  • So can handle a node failure by recomputing lineage up to partitions that can be read from RAM/Disk/replicated storage.

  • Q: Can Spark handle network partitions?

  • A: Nodes that cannot communicate with scheduler will appear dead. The part of the network that can be reached from scheduler can continue computation, as long as it has enough data to start the lineage from (if all replicas of a required partition cannot be reached, cluster cannot make progress)

  • What happens when there isn't enough memory?

    • LRU (Least Recently Used) on partitions
      • first on non-persisted
      • then persisted (but they will be available on disk. makes sure user cannot overbook RAM)
    • user can have control on order of eviction via "persistence priority"
    • no reason not to discard non-persisted partitions (if they've already been used)
  • Shouldn't throw away partitions in RAM that are required but hadn't been used.

Eval

  • Degrades to "almost" MapReduce behavior

    • In figure 7, k-means on 100 Hadoop nodes takes 76-80 seconds
      • Why is Spark faster in this fig? persist() -> no re-read
      • How fast if Spark isn't allowed to cache? Figure 12...
    • In figure 12, k-means on 25 Spark nodes (with no partitions allowed in memory) takes 68.8
    • Difference could be because MapReduce would use replicated storage after reduce, but Spark by default would only spill to local disk, no network latency and I/O load on replicas.
    • no architectural reason why Spark would be slower than MR
  • In Figure 7: a couple of weird things

    • 100 GB over 100 EC2 machines
      • 4 420 GB disks ~ 400 MB/s read bw per node
      • 1 GB/machine in 46-82s = 3-6 MB/s/disk
      • Pretty good? Lower than MR paper per disk.
      • Interference from other tenants?
    • Also: 46s + 33s = 79s ~= 82s
      • So, if 33s of compute and 46s of IO, why aren't they overlapped?
    • Not sure on this - a bit hard to tell, since all this partly depends on demand coming from the "sink".
  • Spark assumes it runs on an isolated memory space (multiple schedulers don't share the memory pool well).

  • Can be solved using a "unified memory manager"

  • Note that when there is reuse of RDDs between jobs, they need to run on the same scheduler to benefit anyway.

(from [P09]) Why not just use parallel databases? Commercially available: "Teradata, Aster Data, Netezza, DATAllegro (and therefore soon Microsoft SQL Server via Project Madison), Dataupia, Vertica, ParAccel, Neoview, Greenplum, DB2 (via the Database Partitioning Feature), and Oracle (via Exadata)"

  • At the time, Parallel DBMS were

    • Some are expensive and Hard to set up right
    • SQL declarative (vs. procedural)
    • Required schema, indices etc (an advantages in some cases)
    • "Not made here"
  • Picollo [P10] uses snapshots of a distributed key-value store to handle fault tolerance.

    • Computation is comprised of control functions and kernel functions.
    • Control functions are responsible for setting up tables (also locality), launching kernels, synchronization (barriers that wait for all kernels to complete), and starting checkpoints
    • Kernels use the key value store. There is a function to merge conflicting writes.
    • Checkpoints using Chandy-Lamport
    • all data has to fit in RAM
    • to recover, all nodes need to revert (expensive)
    • no way to mitigate stragglers, cannot just re-run a kernel without reverting to a snapshot

[P09] "A Comparison of Approaches to Large-Scale Data Analysis", Pavlo et al. SIGMOD'09

[P10] Piccolo: Building Fast, Distributed Programs with Partitioned Tables, Power and Li, OSDI'10