SlideShare ist ein Scribd-Unternehmen logo
1 von 48
Downloaden Sie, um offline zu lesen
Lightning-Fast Cluster Computing
with Spark and Shark
Mayuresh Kunjir and Harold Lim
Duke University
Outline
• Spark
– Spark Overview
– Components
– Life of a Job
– Spark Deployment
• Shark
– Motivation
– Architecture
• Results and Live Demo
Spark Overview
• Open source cluster computing system that aims
to make data analytics fast
– Supports diverse workloads
– sub-second latency
– fault tolerance
– Simplicity
• Research Paper: Resilient Distributed Datasets: A
Fault-Tolerant Abstraction for In-Memory Cluster
Computing [Zaharia et al., NSDI 2012]
Small Codebase
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
Hadoop I/O:
400 LOC
Mesos backend:
700 LOC
Standalone backend:
1700 LOC
Interpreter:
3300 LOC
Spark core: 16,000 LOC
Operators: 2000
Block manager: 2700
Scheduler: 2500
Networking: 1200
Accumulators: 200 Broadcast: 3500
Components
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
sc = new SparkContext
f = sc.textFile(“…”)
f.filter(…)
.count()
...
Your program
Spark client
(app master) Spark worker
HDFS, HBase, …
Block
manager
Task
threads
RDD graph
Scheduler
Block tracker
Shuffle tracker
Cluster
manager
Spark Program
• Can be written using Scala, Java, or Python.
• Spark includes spark-shell to run spark
interactively
• There is also a higher-level abstraction called
Shark (explained in the 2nd half of talk) that
exposes HiveQL language and compiles down to
Spark program
• Latest release of Spark can be downloaded from
spark-project.org/downloads.
– Includes examples, e.g., K-means, logistic regression,
alternating least squares matrix factorization, etc
RDD
• A Spark program revolves around the concept
of resilient distributed datasets (RDD)
– Fault-tolerant collection of elements that can be
operated on in parallel
– Perform operations on RDD
• Transformations (e.g., map, flatMap, union, filter, etc)
that creates new RDD
• Actions returns a value to the driver program (e.g.,
collect, count, etc)
Example Program
• val sc = new SparkContext(
“spark://...”, “MyJob”, home, jars)
val file = sc.textFile(“hdfs://...”)
val errors = file.filter(_.contains(“ERROR”))
errors.cache()
errors.count()
Resilient distributed
datasets (RDDs)
Action
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
RDD Graph
• First run: data not in
cache, so use
HadoopRDD’s locality
prefs (from HDFS)
• Second run: FilteredRDD
is in cache, so use its
locations
• If something falls out of
cache, go back to HDFS
HadoopRDD
path = hdfs://...
FilteredRDD
func = _.contains(…)
shouldCache = true
file:
errors:
Dataset-level view:
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
Scheduling Process
rdd1.join(rdd2)
.groupBy(…)
.filter(…)
RDD Objects
build operator DAG
agnostic to
operators!
doesn’t know
about stages
DAGScheduler
split graph into
stages of tasks
submit each
stage as ready
DAG
TaskScheduler
TaskSet
launch tasks via
cluster manager
retry failed or
straggling tasks
Cluster
manager
Worker
execute tasks
store and serve
blocks
Block
manager
Threads
Task
stage
failed
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
RDD Abstractions
• Extensible (Can implement new RDD operations,
e.g., to read from different sources)
• The current implemented RDD operations can
support a wide-range of workloads
• The RDD Interface
– Set of partitions (“splits”)
– List of dependencies on parent RDDs
– Function to compute a partition given parents
– Optional preferred locations
– Optional partitioning info (Partitioner)
Example: JoinedRDD
• partitions = one per reduce task
• dependencies = “shuffle” on each parent
• compute(partition) = read and join shuffled
data
• preferredLocations(part) = none
• partitioner = HashPartitioner(numTasks)
Spark will now know
this data is hashed!
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
Dependency Types
• Unlike Hadoop, supports a wide range of
dependency between operations
union
groupByKey
join with inputs
not co-partitioned
join with
inputs co-
partitioned
map, filter
“Narrow” deps: “Wide” (shuffle) deps:
DAG Scheduler Optimizations
Pipelines narrow ops.
within a stage
Picks join algorithms
based on partitioning
(minimize shuffles)
Reuses previously
cached data join
union
groupBy
map
Stage 3
Stage 1
Stage 2
A: B:
C: D:
E:
F:
G:
= previously computed partition
Task
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
Task Details
• Each Task object is self-contained
– Contains all transformation code up to input
boundary (e.g. HadoopRDD => filter => map)
• Allows Tasks on cached data even if they fall out
of cache
Design goal: any Task can run on any node
Only way a Task can fail is lost map output files
• Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
TaskScheduler Details
• Can run multiple concurrent TaskSets (Stages),
but currently does so in FIFO order
– Would be really easy to plug in other policies!
• Responsible for scheduling and launching
tasks on Worker nodes
• We (Duke) have implemented a Fair Scheduler
Worker
• Implemented by the Executor class
• Receives self-contained Task objects and calls run() on
them in a thread pool
• Tasks share the same JVM, which allows launching new
tasks quickly
• Has a BlockManager for serving shuffle data and
cachedRDDs (uses the same JVM memory space)
• CachedRDD are configurable
– can be stored as Java object (no
serialization/deserialization overhead) or Serialized
objects.
– Whether to spill to disk or recompute partitions from
parent RDDs when data fall out of cache
– LRU eviction policy
Spark Deployment
• Spark with Mesos (fine-grained)
– Incubator.apache.org/mesos
– Mesos offers resources to Spark programs (using
some configurable policy)
– Each spark tasks run as separate Mesos tasks
• Spark with Mesos (Coarse-grained)
– Only 1 Mesos task is launched on each machine
– Mesos Tasks are long-running and released after
program has completed
– Spark program bypasses Mesos scheduler and
dynamically schedules spark tasks on Mesos tasks (can
schedule more spark tasks on a Mesos task)
Spark Deployment
• Spark Stand-alone Mode
– Similar to Mesos Coarse-grained mode
– No need to have Mesos running on the cluster
• Spark with YARN (NextGen Hadoop)
– Requests pre-defined number of resource
containers from YARN
– Holds on to resource containers until the entire
Spark program finishes
– Spark schedules which tasks gets run on the
obtained resource containers
Another Example Spark Programval sc = new SparkContext(args(0), "SparkLocalKMeans",home,jars)
val lines = sc.textFile(args(1))
val data = lines.map(parseVector _).cache()
val K = args(2).toInt
val convergeDist = args(3).toDouble
var kPoints = data.takeSample(false, K, 42).toArray
var tempDist = 1.0
while(tempDist > convergeDist) {
var closest = data.map (p => (closestPoint(p, kPoints), (p, 1)))
var pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) =>
(x1 + x2, y1 + y2)}
var newPoints = pointStats.map {pair =>
(pair._1, pair._2._1 / pair._2._2)}.collectAsMap()
tempDist = 0.0
for (i <- 0 until K) {
tempDist += kPoints(i).squaredDist(newPoints(i))
}
for (newP <- newPoints) {
kPoints(newP._1) = newP._2
}
println("Finished iteration (delta = " + tempDist + ")")
}
println("Final centers:")
kPoints.foreach(println)
Other Spark Features: Shared Variables
• Normally, Spark operations work on separate copies of all
variables
• Spark now has support for limited type of read-write
shared variables across tasks:
– Broadcast variables: Keep a read-only variable cached on each
machine (no need to ship a copy of variable with tasks)
• E.g., Give every node a copy of a large input dataset in efficient
manner
• Spark uses efficient broadcast algorithms
– Accumulators: variables that are only “added” to through an
associative operation.
• E.g., To implement counters or sums
• Tasks can add to the accumulator value and the driver program can
read the value
Some Issues
• RDDs cannot be shared across different Spark Programs
– Others have implemented a “server” program/shell that
maintains a long-lived SparkContext (Spark Program) and users
submits queries to this server
– Shark has a server mode
• Task operations can be memory-intensive and cause GC
problems
– Unlike Hadoop, task’s input are put into memory (e.g., grouping
is done using in-memory hash table)
• Base on experience, GC problems can result in poor
performance
– Have to ensure level of parallelism is high enough
– Ensure enough memory partition is set for tasks’ working set
(spark.storage.memoryFraction)
Outline
• Spark
– Spark Overview
– Components
– Life of a Job
– Spark Deployment
• Shark
– Motivation
– Architecture
• Results and Live Demo
Apache Hive
• Data warehouse over Hadoop developed at
Facebook
• SQL-like language, HiveQL interface to query
structured data on HDFS
• Queries compile to Hadoop MapReduce jobs
• Very popular: 90+% of Facebook Hadoop jobs
generated by Hive
Hive Architecture
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Hive Principles
• SQL provides a familiar interface for users
• Extensible types, functions, and storage
formats
• Horizontally scalable with high performance
on large datasets
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Hive Downsides
• Not interactive
– Hadoop startup latency is ~20 seconds, even for
small jobs
• No query locality
– If queries operate on the same subset of data,
they still run from scratch
– Reading data from disk is often bottleneck
• Requires separate machine learning dataflow
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Shark Motivations
• Data warehouses exhibit a huge amount of
temporal locality
– 90% of Facebook queries could be served in RAM
• Can we keep all the benefits of Hive
(scalability and extensibility) and exploit the
temporal locality?
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Hive
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Shark
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Introducing Shark
• Shark = Spark +
Hive
• Run HiveQL queries through Spark with Hive
UDF, UDAF, SerDe
• Utilize Spark’s in-memory RDD caching and
flexible language capabilities
• Integrates with Spark for machine learning
operations
Borrowed from Spark User Meetup, February 2012, “Shark – Hive on Spark”
Caching Data in Shark
• Creates a table cached in a cluster’s memory using
RDD.cache()
CREATE TABLE mytable_cached AS SELECT *
from mytable WHERE count > 10;
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Example: Log Mining
• Load error messages from a log into memory, then
interactively search for various patterns
Spark: lines = spark.textFile(“hdfs://...”)
errors = lines.filter(_.startsWith(“ERROR”))
messages = errors.map(_.split(‘t’)(1))
messages.cache()
messages.filter(_.contains(“foo”)).count
messages.filter(_.contains(“bar”)).count
CREATE TABLE log(header string, message string) ROW FORMAT
DELIMITED FIELDS TERMINATED BY ‘t’ LOCATION “hdfs://...”;
CREATE TABLE errors_cached AS SELECT message FROM log WHERE
header == “ERROR”;
SELECT count(*) FROM errors_cached WHERE message LIKE “%foo%”;
SELECT count(*) FROM errors_cached WHERE message LIKE “%bar%”;
Shark:
Borrowed from Spark User Meetup, February 2012, “Shark – Hive on Spark”
Data Model
• Tables: unit of data with the same schema
• Partitions: e.g. range-partition tables by date
• Buckets: hash partitions within partitions
– not yet supported in Shark
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Data Types
• Primitive types
– TINYINT, SMALLINT, INT, BIGINT
– BOOLEAN
– FLOAT, DOUBLE
– STRING
• Complex types
– Structs: STRUCT {a INT; b INT}
– Arrays: [‘a’, ‘b’, ‘c’]
– Maps (key-value pairs): M[‘key’]
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
HiveQL
• Subset of SQL
– Projection, Selection
– Group-by and aggregations
– Sort by and order by
– Joins
– Sub queries, unions
• Hive-specific
– Supports custom map/reduce scripts (TRANSFORM)
– Hints for performance optimizations
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Performance Optimizations
• Caching data in-memory
• Hash-based shuffles for group-by
• Push-down of limits
• Join optimizations through Partial DAG
Execution
• Columnar memory storage
Caching
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Sort, limit, hash shuffle
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Outline
• Spark
– Spark Overview
– Components
– Life of a Job
– Spark Deployment
• Shark
– Motivation
– Architecture
• Results and Live Demo
TPC-H Data
• 5 node cluster running Hive 0.9 and Shark 0.2
• 50GB data on HDFS
• Data read as Hive external tables
Hive versus Shark
Query On Hive On Shark (disk)
1 0:06:10 0:02:20
2 0:10:00 0:07:30
3 0:14:00 0:05:10
4 0:11:40 0:04:30
5 0:17:30 0:07:20
6 0:03:10 0:01:35
7 0:29:10 0:17:40
8 0:19:10 0:09:50
9 0:48:20 0:19:45
10 0:15:00 0:03:50
11 0:07:30 0:02:00
12 0:10:30 0:06:20
13 0:10:00 0:04:00
14 0:05:35 0:01:50
15 0:07:30 0:01:40
16 0:12:50 0:04:50
17 0:20:00 0:10:30
18 0:30:00 0:17:10
19 0:11:40 0:07:05
20 0:15:00 0:04:10
21 0:36:40 0:19:15
22 0:10:10 0:03:40
Number of reducers
have to be explicitly
set in Shark
Performance Tuning
• Two parameters that can significantly affect
performance:
1. Setting the number of reducers
2. Map-side aggregation
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Number of Reducers
• SET mapred.reduce.tasks = 50;
• Shark relies on Spark to infer the number of
map tasks (automatically based on input size)
• Number of reduce tasks need to be specified
by the user
• Out of memory error on slaves if num too
small
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Map-side Aggregation
• SET hive.map.aggr = TRUE;
• Aggregation functions are algebraic and can
be applied on mappers to reduce shuffle data
• Each mapper builds a hash-table to do the first
–level aggregation
Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
Possible Improvements
• Caching is currently explicitly set
– Can this be set automatically?
• Multi-query optimization
– What to cache?
• Treating workload as a sequence
– When to cache?
– When to run a query?
• Notion of Fairness
– Is the notion of Hadoop fairness still valid, given that Spark can also
utilize memory (cached RDD) resources?
• Better support for Multi-tenancy?
– Spark was originally designed/implemented to have each user
workload as separate Spark program
– However, RDDs can’t be shared across different Spark Programs
– Current workaround: Have a single Spark program server and
implement a fair task scheduler
– Is this good enough?
Useful Links
• Project home pages
– http://spark-project.org/
– http://shark.cs.berkeley.edu/
• Research Papers
– Resilient Distributed Datasets: A Fault-Tolerant Abstraction for
In-Memory Cluster Computing. Matei Zaharia, Mosharaf
Chowdhury, Tathagata Das, Ankur Dave, Justin Ma, Murphy
McCauley, Michael J. Franklin, Scott Shenker, Ion Stoica. NSDI
2012. April 2012.
– Shark: SQL and Rich Analytics at Scale. Reynold Xin, Joshua
Rosen, Matei Zaharia, Michael J. Franklin, Scott Shenker, Ion
Stoica. Technical Report UCB/EECS-2012-214. November 2012.
• AMP Camp – Big Data Bootcamp
– http://ampcamp.berkeley.edu/amp-camp-one-berkeley-2012/
Questions?
Thank you!
• mayuresh@cs.duke.edu
• harold@cs.duke.edu

Weitere ähnliche Inhalte

Was ist angesagt?

Introduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkIntroduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkTaras Matyashovsky
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overviewDataArt
 
DTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsDTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsCheng Lian
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introductionsudhakara st
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsAnton Kirillov
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDsDean Chen
 
Intro to apache spark stand ford
Intro to apache spark stand fordIntro to apache spark stand ford
Intro to apache spark stand fordThu Hiền
 
Introduction to spark
Introduction to sparkIntroduction to spark
Introduction to sparkHome
 
Introduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemIntroduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemBojan Babic
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekVenkata Naga Ravi
 
Apache Spark - Intro to Large-scale recommendations with Apache Spark and Python
Apache Spark - Intro to Large-scale recommendations with Apache Spark and PythonApache Spark - Intro to Large-scale recommendations with Apache Spark and Python
Apache Spark - Intro to Large-scale recommendations with Apache Spark and PythonChristian Perone
 
Apache spark sneha challa- google pittsburgh-aug 25th
Apache spark  sneha challa- google pittsburgh-aug 25thApache spark  sneha challa- google pittsburgh-aug 25th
Apache spark sneha challa- google pittsburgh-aug 25thSneha Challa
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark TutorialAhmet Bulut
 

Was ist angesagt? (19)

Introduction to real time big data with Apache Spark
Introduction to real time big data with Apache SparkIntroduction to real time big data with Apache Spark
Introduction to real time big data with Apache Spark
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overview
 
DTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsDTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime Internals
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introduction
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
 
Intro to apache spark stand ford
Intro to apache spark stand fordIntro to apache spark stand ford
Intro to apache spark stand ford
 
Introduction to spark
Introduction to sparkIntroduction to spark
Introduction to spark
 
Introduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemIntroduction to Apache Spark Ecosystem
Introduction to Apache Spark Ecosystem
 
Apache spark core
Apache spark coreApache spark core
Apache spark core
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
Apache Spark on HDinsight Training
Apache Spark on HDinsight TrainingApache Spark on HDinsight Training
Apache Spark on HDinsight Training
 
Apache Spark - Intro to Large-scale recommendations with Apache Spark and Python
Apache Spark - Intro to Large-scale recommendations with Apache Spark and PythonApache Spark - Intro to Large-scale recommendations with Apache Spark and Python
Apache Spark - Intro to Large-scale recommendations with Apache Spark and Python
 
Internals
InternalsInternals
Internals
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Scala and spark
Scala and sparkScala and spark
Scala and spark
 
Apache spark sneha challa- google pittsburgh-aug 25th
Apache spark  sneha challa- google pittsburgh-aug 25thApache spark  sneha challa- google pittsburgh-aug 25th
Apache spark sneha challa- google pittsburgh-aug 25th
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 

Ähnlich wie TriHUG talk on Spark and Shark

Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Olalekan Fuad Elesin
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkRahul Jain
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Djamel Zouaoui
 
Fast Data Analytics with Spark and Python
Fast Data Analytics with Spark and PythonFast Data Analytics with Spark and Python
Fast Data Analytics with Spark and PythonBenjamin Bengfort
 
NYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / SolrNYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / Solrthelabdude
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoMapR Technologies
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
Dec6 meetup spark presentation
Dec6 meetup spark presentationDec6 meetup spark presentation
Dec6 meetup spark presentationRamesh Mudunuri
 
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability Meetup
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability MeetupApache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability Meetup
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability MeetupHyderabad Scalability Meetup
 
Why Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraWhy Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraHandaru Sakti
 
Spark from the Surface
Spark from the SurfaceSpark from the Surface
Spark from the SurfaceJosi Aranda
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark FundamentalsZahra Eskandari
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark Mostafa
 
ApacheCon NA 2015 Spark / Solr Integration
ApacheCon NA 2015 Spark / Solr IntegrationApacheCon NA 2015 Spark / Solr Integration
ApacheCon NA 2015 Spark / Solr Integrationthelabdude
 
Unit II Real Time Data Processing tools.pptx
Unit II Real Time Data Processing tools.pptxUnit II Real Time Data Processing tools.pptx
Unit II Real Time Data Processing tools.pptxRahul Borate
 
Big_data_analytics_NoSql_Module-4_Session
Big_data_analytics_NoSql_Module-4_SessionBig_data_analytics_NoSql_Module-4_Session
Big_data_analytics_NoSql_Module-4_SessionRUHULAMINHAZARIKA
 

Ähnlich wie TriHUG talk on Spark and Shark (20)

Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming
 
Fast Data Analytics with Spark and Python
Fast Data Analytics with Spark and PythonFast Data Analytics with Spark and Python
Fast Data Analytics with Spark and Python
 
Apache Spark Core
Apache Spark CoreApache Spark Core
Apache Spark Core
 
NYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / SolrNYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / Solr
 
Spark core
Spark coreSpark core
Spark core
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of Twingo
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
Dec6 meetup spark presentation
Dec6 meetup spark presentationDec6 meetup spark presentation
Dec6 meetup spark presentation
 
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability Meetup
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability MeetupApache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability Meetup
Apache Spark - Lightning Fast Cluster Computing - Hyderabad Scalability Meetup
 
APACHE SPARK.pptx
APACHE SPARK.pptxAPACHE SPARK.pptx
APACHE SPARK.pptx
 
Why Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraWhy Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data Era
 
Spark from the Surface
Spark from the SurfaceSpark from the Surface
Spark from the Surface
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark
 
ApacheCon NA 2015 Spark / Solr Integration
ApacheCon NA 2015 Spark / Solr IntegrationApacheCon NA 2015 Spark / Solr Integration
ApacheCon NA 2015 Spark / Solr Integration
 
Unit II Real Time Data Processing tools.pptx
Unit II Real Time Data Processing tools.pptxUnit II Real Time Data Processing tools.pptx
Unit II Real Time Data Processing tools.pptx
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
Big_data_analytics_NoSql_Module-4_Session
Big_data_analytics_NoSql_Module-4_SessionBig_data_analytics_NoSql_Module-4_Session
Big_data_analytics_NoSql_Module-4_Session
 

Mehr von trihug

TriHUG October: Apache Ranger
TriHUG October: Apache RangerTriHUG October: Apache Ranger
TriHUG October: Apache Rangertrihug
 
TriHUG Feb: Hive on spark
TriHUG Feb: Hive on sparkTriHUG Feb: Hive on spark
TriHUG Feb: Hive on sparktrihug
 
TriHUG 3/14: HBase in Production
TriHUG 3/14: HBase in ProductionTriHUG 3/14: HBase in Production
TriHUG 3/14: HBase in Productiontrihug
 
TriHUG 2/14: Apache Sentry
TriHUG 2/14: Apache SentryTriHUG 2/14: Apache Sentry
TriHUG 2/14: Apache Sentrytrihug
 
Impala presentation
Impala presentationImpala presentation
Impala presentationtrihug
 
Practical pig
Practical pigPractical pig
Practical pigtrihug
 
Financial services trihug
Financial services trihugFinancial services trihug
Financial services trihugtrihug
 
TriHUG January 2012 Talk by Chris Shain
TriHUG January 2012 Talk by Chris ShainTriHUG January 2012 Talk by Chris Shain
TriHUG January 2012 Talk by Chris Shaintrihug
 
TriHUG November HCatalog Talk by Alan Gates
TriHUG November HCatalog Talk by Alan GatesTriHUG November HCatalog Talk by Alan Gates
TriHUG November HCatalog Talk by Alan Gatestrihug
 
TriHUG November Pig Talk by Alan Gates
TriHUG November Pig Talk by Alan GatesTriHUG November Pig Talk by Alan Gates
TriHUG November Pig Talk by Alan Gatestrihug
 
MapR, Implications for Integration
MapR, Implications for IntegrationMapR, Implications for Integration
MapR, Implications for Integrationtrihug
 

Mehr von trihug (11)

TriHUG October: Apache Ranger
TriHUG October: Apache RangerTriHUG October: Apache Ranger
TriHUG October: Apache Ranger
 
TriHUG Feb: Hive on spark
TriHUG Feb: Hive on sparkTriHUG Feb: Hive on spark
TriHUG Feb: Hive on spark
 
TriHUG 3/14: HBase in Production
TriHUG 3/14: HBase in ProductionTriHUG 3/14: HBase in Production
TriHUG 3/14: HBase in Production
 
TriHUG 2/14: Apache Sentry
TriHUG 2/14: Apache SentryTriHUG 2/14: Apache Sentry
TriHUG 2/14: Apache Sentry
 
Impala presentation
Impala presentationImpala presentation
Impala presentation
 
Practical pig
Practical pigPractical pig
Practical pig
 
Financial services trihug
Financial services trihugFinancial services trihug
Financial services trihug
 
TriHUG January 2012 Talk by Chris Shain
TriHUG January 2012 Talk by Chris ShainTriHUG January 2012 Talk by Chris Shain
TriHUG January 2012 Talk by Chris Shain
 
TriHUG November HCatalog Talk by Alan Gates
TriHUG November HCatalog Talk by Alan GatesTriHUG November HCatalog Talk by Alan Gates
TriHUG November HCatalog Talk by Alan Gates
 
TriHUG November Pig Talk by Alan Gates
TriHUG November Pig Talk by Alan GatesTriHUG November Pig Talk by Alan Gates
TriHUG November Pig Talk by Alan Gates
 
MapR, Implications for Integration
MapR, Implications for IntegrationMapR, Implications for Integration
MapR, Implications for Integration
 

Kürzlich hochgeladen

Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticscarlostorres15106
 
Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfAlex Barbosa Coqueiro
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machinePadma Pradeep
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationRidwan Fadjar
 
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks..."LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...Fwdays
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr LapshynFwdays
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupFlorian Wilhelm
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):comworks
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 3652toLead Limited
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piececharlottematthew16
 
Powerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time ClashPowerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time Clashcharlottematthew16
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLScyllaDB
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024The Digital Insurer
 
Vertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsVertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsMiki Katsuragi
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Manik S Magar
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesZilliz
 

Kürzlich hochgeladen (20)

Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
 
Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdf
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machine
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 Presentation
 
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptxE-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
 
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks..."LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
DMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special EditionDMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special Edition
 
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project Setup
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piece
 
Powerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time ClashPowerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time Clash
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQL
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024
 
Vertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsVertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering Tips
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector Databases
 

TriHUG talk on Spark and Shark

  • 1. Lightning-Fast Cluster Computing with Spark and Shark Mayuresh Kunjir and Harold Lim Duke University
  • 2. Outline • Spark – Spark Overview – Components – Life of a Job – Spark Deployment • Shark – Motivation – Architecture • Results and Live Demo
  • 3. Spark Overview • Open source cluster computing system that aims to make data analytics fast – Supports diverse workloads – sub-second latency – fault tolerance – Simplicity • Research Paper: Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing [Zaharia et al., NSDI 2012]
  • 4. Small Codebase • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals Hadoop I/O: 400 LOC Mesos backend: 700 LOC Standalone backend: 1700 LOC Interpreter: 3300 LOC Spark core: 16,000 LOC Operators: 2000 Block manager: 2700 Scheduler: 2500 Networking: 1200 Accumulators: 200 Broadcast: 3500
  • 5. Components • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals sc = new SparkContext f = sc.textFile(“…”) f.filter(…) .count() ... Your program Spark client (app master) Spark worker HDFS, HBase, … Block manager Task threads RDD graph Scheduler Block tracker Shuffle tracker Cluster manager
  • 6. Spark Program • Can be written using Scala, Java, or Python. • Spark includes spark-shell to run spark interactively • There is also a higher-level abstraction called Shark (explained in the 2nd half of talk) that exposes HiveQL language and compiles down to Spark program • Latest release of Spark can be downloaded from spark-project.org/downloads. – Includes examples, e.g., K-means, logistic regression, alternating least squares matrix factorization, etc
  • 7. RDD • A Spark program revolves around the concept of resilient distributed datasets (RDD) – Fault-tolerant collection of elements that can be operated on in parallel – Perform operations on RDD • Transformations (e.g., map, flatMap, union, filter, etc) that creates new RDD • Actions returns a value to the driver program (e.g., collect, count, etc)
  • 8. Example Program • val sc = new SparkContext( “spark://...”, “MyJob”, home, jars) val file = sc.textFile(“hdfs://...”) val errors = file.filter(_.contains(“ERROR”)) errors.cache() errors.count() Resilient distributed datasets (RDDs) Action • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 9. RDD Graph • First run: data not in cache, so use HadoopRDD’s locality prefs (from HDFS) • Second run: FilteredRDD is in cache, so use its locations • If something falls out of cache, go back to HDFS HadoopRDD path = hdfs://... FilteredRDD func = _.contains(…) shouldCache = true file: errors: Dataset-level view: • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 10. Scheduling Process rdd1.join(rdd2) .groupBy(…) .filter(…) RDD Objects build operator DAG agnostic to operators! doesn’t know about stages DAGScheduler split graph into stages of tasks submit each stage as ready DAG TaskScheduler TaskSet launch tasks via cluster manager retry failed or straggling tasks Cluster manager Worker execute tasks store and serve blocks Block manager Threads Task stage failed • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 11. RDD Abstractions • Extensible (Can implement new RDD operations, e.g., to read from different sources) • The current implemented RDD operations can support a wide-range of workloads • The RDD Interface – Set of partitions (“splits”) – List of dependencies on parent RDDs – Function to compute a partition given parents – Optional preferred locations – Optional partitioning info (Partitioner)
  • 12. Example: JoinedRDD • partitions = one per reduce task • dependencies = “shuffle” on each parent • compute(partition) = read and join shuffled data • preferredLocations(part) = none • partitioner = HashPartitioner(numTasks) Spark will now know this data is hashed! • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 13. Dependency Types • Unlike Hadoop, supports a wide range of dependency between operations union groupByKey join with inputs not co-partitioned join with inputs co- partitioned map, filter “Narrow” deps: “Wide” (shuffle) deps:
  • 14. DAG Scheduler Optimizations Pipelines narrow ops. within a stage Picks join algorithms based on partitioning (minimize shuffles) Reuses previously cached data join union groupBy map Stage 3 Stage 1 Stage 2 A: B: C: D: E: F: G: = previously computed partition Task • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 15. Task Details • Each Task object is self-contained – Contains all transformation code up to input boundary (e.g. HadoopRDD => filter => map) • Allows Tasks on cached data even if they fall out of cache Design goal: any Task can run on any node Only way a Task can fail is lost map output files • Borrowed from Spark User Meetup 2012, Introduction to Spark Internals
  • 16. TaskScheduler Details • Can run multiple concurrent TaskSets (Stages), but currently does so in FIFO order – Would be really easy to plug in other policies! • Responsible for scheduling and launching tasks on Worker nodes • We (Duke) have implemented a Fair Scheduler
  • 17. Worker • Implemented by the Executor class • Receives self-contained Task objects and calls run() on them in a thread pool • Tasks share the same JVM, which allows launching new tasks quickly • Has a BlockManager for serving shuffle data and cachedRDDs (uses the same JVM memory space) • CachedRDD are configurable – can be stored as Java object (no serialization/deserialization overhead) or Serialized objects. – Whether to spill to disk or recompute partitions from parent RDDs when data fall out of cache – LRU eviction policy
  • 18. Spark Deployment • Spark with Mesos (fine-grained) – Incubator.apache.org/mesos – Mesos offers resources to Spark programs (using some configurable policy) – Each spark tasks run as separate Mesos tasks • Spark with Mesos (Coarse-grained) – Only 1 Mesos task is launched on each machine – Mesos Tasks are long-running and released after program has completed – Spark program bypasses Mesos scheduler and dynamically schedules spark tasks on Mesos tasks (can schedule more spark tasks on a Mesos task)
  • 19. Spark Deployment • Spark Stand-alone Mode – Similar to Mesos Coarse-grained mode – No need to have Mesos running on the cluster • Spark with YARN (NextGen Hadoop) – Requests pre-defined number of resource containers from YARN – Holds on to resource containers until the entire Spark program finishes – Spark schedules which tasks gets run on the obtained resource containers
  • 20. Another Example Spark Programval sc = new SparkContext(args(0), "SparkLocalKMeans",home,jars) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt val convergeDist = args(3).toDouble var kPoints = data.takeSample(false, K, 42).toArray var tempDist = 1.0 while(tempDist > convergeDist) { var closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) var pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)} var newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap() tempDist = 0.0 for (i <- 0 until K) { tempDist += kPoints(i).squaredDist(newPoints(i)) } for (newP <- newPoints) { kPoints(newP._1) = newP._2 } println("Finished iteration (delta = " + tempDist + ")") } println("Final centers:") kPoints.foreach(println)
  • 21. Other Spark Features: Shared Variables • Normally, Spark operations work on separate copies of all variables • Spark now has support for limited type of read-write shared variables across tasks: – Broadcast variables: Keep a read-only variable cached on each machine (no need to ship a copy of variable with tasks) • E.g., Give every node a copy of a large input dataset in efficient manner • Spark uses efficient broadcast algorithms – Accumulators: variables that are only “added” to through an associative operation. • E.g., To implement counters or sums • Tasks can add to the accumulator value and the driver program can read the value
  • 22. Some Issues • RDDs cannot be shared across different Spark Programs – Others have implemented a “server” program/shell that maintains a long-lived SparkContext (Spark Program) and users submits queries to this server – Shark has a server mode • Task operations can be memory-intensive and cause GC problems – Unlike Hadoop, task’s input are put into memory (e.g., grouping is done using in-memory hash table) • Base on experience, GC problems can result in poor performance – Have to ensure level of parallelism is high enough – Ensure enough memory partition is set for tasks’ working set (spark.storage.memoryFraction)
  • 23. Outline • Spark – Spark Overview – Components – Life of a Job – Spark Deployment • Shark – Motivation – Architecture • Results and Live Demo
  • 24. Apache Hive • Data warehouse over Hadoop developed at Facebook • SQL-like language, HiveQL interface to query structured data on HDFS • Queries compile to Hadoop MapReduce jobs • Very popular: 90+% of Facebook Hadoop jobs generated by Hive
  • 25. Hive Architecture Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 26. Hive Principles • SQL provides a familiar interface for users • Extensible types, functions, and storage formats • Horizontally scalable with high performance on large datasets Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 27. Hive Downsides • Not interactive – Hadoop startup latency is ~20 seconds, even for small jobs • No query locality – If queries operate on the same subset of data, they still run from scratch – Reading data from disk is often bottleneck • Requires separate machine learning dataflow Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 28. Shark Motivations • Data warehouses exhibit a huge amount of temporal locality – 90% of Facebook queries could be served in RAM • Can we keep all the benefits of Hive (scalability and extensibility) and exploit the temporal locality? Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 29. Hive Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 30. Shark Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 31. Introducing Shark • Shark = Spark + Hive • Run HiveQL queries through Spark with Hive UDF, UDAF, SerDe • Utilize Spark’s in-memory RDD caching and flexible language capabilities • Integrates with Spark for machine learning operations Borrowed from Spark User Meetup, February 2012, “Shark – Hive on Spark”
  • 32. Caching Data in Shark • Creates a table cached in a cluster’s memory using RDD.cache() CREATE TABLE mytable_cached AS SELECT * from mytable WHERE count > 10; Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 33. Example: Log Mining • Load error messages from a log into memory, then interactively search for various patterns Spark: lines = spark.textFile(“hdfs://...”) errors = lines.filter(_.startsWith(“ERROR”)) messages = errors.map(_.split(‘t’)(1)) messages.cache() messages.filter(_.contains(“foo”)).count messages.filter(_.contains(“bar”)).count CREATE TABLE log(header string, message string) ROW FORMAT DELIMITED FIELDS TERMINATED BY ‘t’ LOCATION “hdfs://...”; CREATE TABLE errors_cached AS SELECT message FROM log WHERE header == “ERROR”; SELECT count(*) FROM errors_cached WHERE message LIKE “%foo%”; SELECT count(*) FROM errors_cached WHERE message LIKE “%bar%”; Shark: Borrowed from Spark User Meetup, February 2012, “Shark – Hive on Spark”
  • 34. Data Model • Tables: unit of data with the same schema • Partitions: e.g. range-partition tables by date • Buckets: hash partitions within partitions – not yet supported in Shark Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 35. Data Types • Primitive types – TINYINT, SMALLINT, INT, BIGINT – BOOLEAN – FLOAT, DOUBLE – STRING • Complex types – Structs: STRUCT {a INT; b INT} – Arrays: [‘a’, ‘b’, ‘c’] – Maps (key-value pairs): M[‘key’] Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 36. HiveQL • Subset of SQL – Projection, Selection – Group-by and aggregations – Sort by and order by – Joins – Sub queries, unions • Hive-specific – Supports custom map/reduce scripts (TRANSFORM) – Hints for performance optimizations Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 37. Performance Optimizations • Caching data in-memory • Hash-based shuffles for group-by • Push-down of limits • Join optimizations through Partial DAG Execution • Columnar memory storage
  • 38. Caching Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 39. Sort, limit, hash shuffle Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 40. Outline • Spark – Spark Overview – Components – Life of a Job – Spark Deployment • Shark – Motivation – Architecture • Results and Live Demo
  • 41. TPC-H Data • 5 node cluster running Hive 0.9 and Shark 0.2 • 50GB data on HDFS • Data read as Hive external tables
  • 42. Hive versus Shark Query On Hive On Shark (disk) 1 0:06:10 0:02:20 2 0:10:00 0:07:30 3 0:14:00 0:05:10 4 0:11:40 0:04:30 5 0:17:30 0:07:20 6 0:03:10 0:01:35 7 0:29:10 0:17:40 8 0:19:10 0:09:50 9 0:48:20 0:19:45 10 0:15:00 0:03:50 11 0:07:30 0:02:00 12 0:10:30 0:06:20 13 0:10:00 0:04:00 14 0:05:35 0:01:50 15 0:07:30 0:01:40 16 0:12:50 0:04:50 17 0:20:00 0:10:30 18 0:30:00 0:17:10 19 0:11:40 0:07:05 20 0:15:00 0:04:10 21 0:36:40 0:19:15 22 0:10:10 0:03:40 Number of reducers have to be explicitly set in Shark
  • 43. Performance Tuning • Two parameters that can significantly affect performance: 1. Setting the number of reducers 2. Map-side aggregation Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 44. Number of Reducers • SET mapred.reduce.tasks = 50; • Shark relies on Spark to infer the number of map tasks (automatically based on input size) • Number of reduce tasks need to be specified by the user • Out of memory error on slaves if num too small Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 45. Map-side Aggregation • SET hive.map.aggr = TRUE; • Aggregation functions are algebraic and can be applied on mappers to reduce shuffle data • Each mapper builds a hash-table to do the first –level aggregation Borrowed from AMP Camp One – Big Data Bootcamp Berkeley, August 2012, “Structured Data with Hive and Shark”
  • 46. Possible Improvements • Caching is currently explicitly set – Can this be set automatically? • Multi-query optimization – What to cache? • Treating workload as a sequence – When to cache? – When to run a query? • Notion of Fairness – Is the notion of Hadoop fairness still valid, given that Spark can also utilize memory (cached RDD) resources? • Better support for Multi-tenancy? – Spark was originally designed/implemented to have each user workload as separate Spark program – However, RDDs can’t be shared across different Spark Programs – Current workaround: Have a single Spark program server and implement a fair task scheduler – Is this good enough?
  • 47. Useful Links • Project home pages – http://spark-project.org/ – http://shark.cs.berkeley.edu/ • Research Papers – Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing. Matei Zaharia, Mosharaf Chowdhury, Tathagata Das, Ankur Dave, Justin Ma, Murphy McCauley, Michael J. Franklin, Scott Shenker, Ion Stoica. NSDI 2012. April 2012. – Shark: SQL and Rich Analytics at Scale. Reynold Xin, Joshua Rosen, Matei Zaharia, Michael J. Franklin, Scott Shenker, Ion Stoica. Technical Report UCB/EECS-2012-214. November 2012. • AMP Camp – Big Data Bootcamp – http://ampcamp.berkeley.edu/amp-camp-one-berkeley-2012/