SlideShare a Scribd company logo
1 of 26
Download to read offline
Apache Spark in Depth
core concepts, architecture & internals
Anton Kirillov Ooyala, Mar 2016
Roadmap
● RDDs
○ Definition
○ Operations
● Execution workflow
○ DAG
○ Stages and tasks
○ Shuffle
● Architecture
○ Components
○ Memory model
● Coding
○ spark-shell
○ building and submitting Spark applications to YARN
Meet Spark
● Generalized framework for distributed data processing (batch, graph, ML)
● Scala collections functional API for manipulating data at scale
● In-memory data caching and reuse across computations
● Applies set of coarse-grained transformations over partitioned data
● Failure recovery relies on lineage to recompute failed tasks
● Supports majority of input formats and integrates with Mesos / YARN
Spark makes data engineers happy
Backup/restore of Cassandra tables in Parquet
def backup(config: Config) {
sc.cassandraTable(config.keyspace, config.table).map(_.toEvent).toDF()
.write.parquet(config.path)
}
def restore(config: Config) {
sqlContext.read.parquet(config.path)
.map(_.toEvent).saveToCassandra(config.keyspace, config.table)
}
Query different data sources to identify discrepancies
sqlContext.sql {
"""
SELECT count()
FROM cassandra_event_rollups
JOIN mongo_event_rollups
ON cassandra_event_rollups.uuid = cassandra_event_rollups.uuid
WHERE cassandra_event_rollups.value != cassandra_event_rollups.value
""".stripMargin
}
Core Concepts
RDD: Resilient Distributed Dataset
● A fault-tolerant, immutable, parallel data structure
● Provides API for
○ manipulating the collection of elements (transformations and materialization)
○ persisting intermediate results in memory for later reuse
○ controlling partitioning to optimize data placement
● Can be created through deterministic operation
○ from storage (distributed file system, database, plain file)
○ from another RDD
● Stores information about parent RDDs
○ for execution optimization and operations pipelining
○ to recompute the data in case of failure
RDD: a developer’s view
● Distributed immutable data + lazily evaluated operations
○ partitioned data + iterator
○ transformations & actions
● An interface defining 5 main properties
a list of partitions (e.g. splits in Hadoop)
def getPartitions: Array[Partition]
a list of dependencies on other RDDs
def getDependencies: Seq[Dependency[_]]
a function for computing each split
def compute(split: Partition, context: TaskContext): Iterator[T]
(optional) a list of preferred locations to compute each split on
def getPreferredLocations(split: Partition): Seq[String] = Nil
(optional) a partitioner for key-value RDDs
val partitioner: Option[Partitioner] = None
lineage
execution optimization
RDDs Example
● HadoopRDD
○ getPartitions = HDFS blocks
○ getDependencies = None
○ compute = load block in memory
○ getPrefferedLocations = HDFS block locations
○ partitioner = None
● MapPartitionsRDD
○ getPartitions = same as parent
○ getDependencies = parent RDD
○ compute = compute parent and apply map()
○ getPrefferedLocations = same as parent
○ partitioner = None
sparkContext.textFile("hdfs://...")
RDD Operations
● Transformations
○ apply user function to every element in a partition (or to the whole partition)
○ apply aggregation function to the whole dataset (groupBy, sortBy)
○ introduce dependencies between RDDs to form DAG
○ provide functionality for repartitioning (repartition, partitionBy)
● Actions
○ trigger job execution
○ used to materialize computation results
● Extra: persistence
○ explicitly store RDDs in memory, on disk or off-heap (cache, persist)
○ checkpointing for truncating RDD lineage
Execution workflow
10
rdd1.join(rdd2)
.groupBy(...)
.filter(...)
splits graph into
stages of tasks
submits each stage
as ready
launches tasks via
cluster manager
retries failed or
struggling tasks
executes tasks
stores and serves
blocks
Code sample: joining aggregated and raw data
//aggregate events after specific date for given campaign
val events = sc.cassandraTable("demo", "event")
.map(_.toEvent)
.filter(event => event.campaignId == campaignId && event.time.isAfter(watermark))
.keyBy(_.eventType)
.reduceByKey(_ + _)
.cache()
//aggregate campaigns by type
val campaigns = sc.cassandraTable("demo", "campaign")
.map(_.toCampaign)
.filter(campaign => campaign.id == campaignId && campaign.time.isBefore(watermark))
.keyBy(_.eventType)
.reduceByKey(_ + _)
.cache()
//joined rollups and raw events
val joinedTotals = campaigns.join(events)
.map { case (key, (campaign, event)) => CampaignTotals(campaign, event) }
.collect()
//count totals separately
val eventTotals = events.map{ case (t, e) => s"$t -> ${e.value}" }.collect()
val campaignTotals = campaigns.map{ case (t, e) => s"$t -> ${e.value}" }.collect()
DAG
Dependency types
● Narrow (pipelineable)
○ each partition of the parent RDD is used by at most
one partition of the child RDD
○ allow for pipelined execution on one cluster node
○ failure recovery is more efficient as only lost parent
partitions need to be recomputed
● Wide (shuffle)
○ multiple child partitions may depend on one parent
partition
○ require data from all parent partitions to be available
and to be shuffled across the nodes
○ if some partition is lost from all the ancestors a
complete recomputation is needed
Stages and Tasks
● Stages breakdown strategy
○ check backwards from final RDD
○ add each “narrow” dependency to
the current stage
○ create new stage when there’s a
shuffle dependency
● Tasks
○ ShuffleMapTask partitions its
input for shuffle
○ ResultTask sends its output to
the driver
Shuffle
● Shuffle Write
○ redistributes data among partitions
and writes files to disk
○ each hash shuffle task creates one
file per “reduce” task (total = MxR)
○ sort shuffle task creates one file
with regions assigned to reducer
○ sort shuffle uses in-memory sorting
with spillover to disk to get final
result
● Shuffle Read
○ fetches the files and applies
reduce() logic
○ if data ordering is needed then it is
sorted on “reducer” side for any
type of shuffle (SPARK-2926)
Sort Shuffle
● Incoming records accumulated
and sorted in memory according
their target partition ids
● Sorted records are written to file
or multiple files if spilled and then
merged
● index file stores offsets of the
data blocks in the data file
● Sorting without deserialization is
possible under certain conditions
(SPARK-7081)
Architecture Recap
● Spark Driver
○ separate process to execute user
applications
○ creates SparkContext to schedule
jobs execution and negotiate with
cluster manager
● Executors
○ run tasks scheduled by driver
○ store computation results in
memory, on disk or off-heap
○ interact with storage systems
● Cluster Manager
○ Mesos
○ YARN
○ Spark Standalone
Spark Components
Spark Components
● SparkContext
○ represents the connection to a Spark cluster, and can be used to create RDDs, accumulators and
broadcast variables on that cluster
● DAGScheduler
○ computes a DAG of stages for each job and submits them to TaskScheduler
○ determines preferred locations for tasks (based on cache status or shuffle files locations) and finds
minimum schedule to run the jobs
● TaskScheduler
○ responsible for sending tasks to the cluster, running them, retrying if there are failures, and mitigating
stragglers
● SchedulerBackend
○ backend interface for scheduling systems that allows plugging in different implementations(Mesos,
YARN, Standalone, local)
● BlockManager
○ provides interfaces for putting and retrieving blocks both locally and remotely into various stores
(memory, disk, and off-heap)
Memory Management in Spark 1.6
● Execution Memory
○ storage for data needed during tasks execution
○ shuffle-related data
● Storage Memory
○ storage of cached RDDs and broadcast variables
○ possible to borrow from execution memory
(spill otherwise)
○ safeguard value is 0.5 of Spark Memory when cached
blocks are immune to eviction
● User Memory
○ user data structures and internal metadata in Spark
○ safeguarding against OOM
● Reserved memory
○ memory needed for running executor itself and not
strictly related to Spark
Workshop
code available @ github.com/datastrophic/spark-workshop
Execution Modes
● spark-shell --master [ local | spark | yarn-client | mesos]
○ launches REPL connected to specified cluster manager
○ always runs in client mode
● spark-submit --master [ local | spark:// | mesos:// | yarn ] spark-job.jar
○ launches assembly jar on the cluster
● Masters
○ local[k] - run Spark locally with K worker threads
○ spark - launches driver app on Spark Standalone installation
○ mesos - driver will spawn executors on Mesos cluster (deploy-mode: client | cluster)
○ yarn - same idea as with Mesos (deploy-mode: client | cluster)
● Deploy Modes
○ client - driver executed as a separate process on the machine where it has been launched and
spawns executors
○ cluster - driver launched as a container using underlying cluster manager
Invocation examples
spark-shell 
--master yarn 
--deploy-mode client 
--executor-cores 1 
--num-executors 2 
--jars /target/spark-workshop.jar 
--conf spark.cassandra.connection.host=cassandra
spark-submit --class io.datastrophic.spark.workshop.ParametrizedApplicationExample 
--master yarn 
--deploy-mode cluster 
--num-executors 2 
--driver-memory 1g 
--executor-memory 1g 
/target/spark-workshop.jar 
--cassandra-host cassandra 
--keyspace demo 
--table event 
--target-dir /workshop/dumps
Live Demo
● spark-shell
● Spark UI
● creating an app with Typesafe Activator
● Spark SQL and DataFrames API
● coding
Coding ideas
● get familiar with API through sample project
○ join data from different storage systems
○ aggregate data with breakdown by date
● play with caching and persistence
● check out join behavior applying different partitioning
● familiarize with Spark UI
● experiment with new DataSet API (since 1.6)
● [ your awesome idea here ]
Questions
@antonkirillov datastrophic.io

More Related Content

What's hot

Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark FundamentalsZahra Eskandari
 
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkBo Yang
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkRahul Jain
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationDatabricks
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsDatabricks
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabApache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabCloudxLab
 
Learn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideLearn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideWhizlabs
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesDatabricks
 
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
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeDatabricks
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkDatabricks
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 

What's hot (20)

Apache Spark 101
Apache Spark 101Apache Spark 101
Apache Spark 101
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper Optimization
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabApache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Learn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideLearn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive Guide
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on Kubernetes
 
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
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
Apache spark
Apache sparkApache spark
Apache spark
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache Spark
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 

Viewers also liked

Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingCloudera, Inc.
 
MapR Tutorial Series
MapR Tutorial SeriesMapR Tutorial Series
MapR Tutorial Seriesselvaraaju
 
MapR and Cisco Make IT Better
MapR and Cisco Make IT BetterMapR and Cisco Make IT Better
MapR and Cisco Make IT BetterMapR Technologies
 
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)Amazon Web Services
 
Hands on MapR -- Viadea
Hands on MapR -- ViadeaHands on MapR -- Viadea
Hands on MapR -- Viadeaviadea
 
Architectural Overview of MapR's Apache Hadoop Distribution
Architectural Overview of MapR's Apache Hadoop DistributionArchitectural Overview of MapR's Apache Hadoop Distribution
Architectural Overview of MapR's Apache Hadoop Distributionmcsrivas
 
MapR M7: Providing an enterprise quality Apache HBase API
MapR M7: Providing an enterprise quality Apache HBase APIMapR M7: Providing an enterprise quality Apache HBase API
MapR M7: Providing an enterprise quality Apache HBase APImcsrivas
 
Apache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterApache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterDatabricks
 
MapR Data Analyst
MapR Data AnalystMapR Data Analyst
MapR Data Analystselvaraaju
 

Viewers also liked (12)

Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer Training
 
MapR Tutorial Series
MapR Tutorial SeriesMapR Tutorial Series
MapR Tutorial Series
 
MapR and Cisco Make IT Better
MapR and Cisco Make IT BetterMapR and Cisco Make IT Better
MapR and Cisco Make IT Better
 
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)
AWS re:Invent 2016: Fraud Detection with Amazon Machine Learning on AWS (FIN301)
 
Apache Spark & Hadoop
Apache Spark & HadoopApache Spark & Hadoop
Apache Spark & Hadoop
 
Modern Data Architecture
Modern Data ArchitectureModern Data Architecture
Modern Data Architecture
 
Hands on MapR -- Viadea
Hands on MapR -- ViadeaHands on MapR -- Viadea
Hands on MapR -- Viadea
 
Architectural Overview of MapR's Apache Hadoop Distribution
Architectural Overview of MapR's Apache Hadoop DistributionArchitectural Overview of MapR's Apache Hadoop Distribution
Architectural Overview of MapR's Apache Hadoop Distribution
 
MapR M7: Providing an enterprise quality Apache HBase API
MapR M7: Providing an enterprise quality Apache HBase APIMapR M7: Providing an enterprise quality Apache HBase API
MapR M7: Providing an enterprise quality Apache HBase API
 
Deep Learning for Fraud Detection
Deep Learning for Fraud DetectionDeep Learning for Fraud Detection
Deep Learning for Fraud Detection
 
Apache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterApache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and Smarter
 
MapR Data Analyst
MapR Data AnalystMapR Data Analyst
MapR Data Analyst
 

Similar to Apache Spark in Depth: Core Concepts, Architecture & Internals

Data processing platforms with SMACK: Spark and Mesos internals
Data processing platforms with SMACK:  Spark and Mesos internalsData processing platforms with SMACK:  Spark and Mesos internals
Data processing platforms with SMACK: Spark and Mesos internalsAnton Kirillov
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkDatio Big Data
 
Apache spark - Installation
Apache spark - InstallationApache spark - Installation
Apache spark - InstallationMartin Zapletal
 
Apache spark - Spark's distributed programming model
Apache spark - Spark's distributed programming modelApache spark - Spark's distributed programming model
Apache spark - Spark's distributed programming modelMartin Zapletal
 
Apache Spark II (SparkSQL)
Apache Spark II (SparkSQL)Apache Spark II (SparkSQL)
Apache Spark II (SparkSQL)Datio Big Data
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型wang xing
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Massimo Schenone
 
Core Services behind Spark Job Execution
Core Services behind Spark Job ExecutionCore Services behind Spark Job Execution
Core Services behind Spark Job Executiondatamantra
 
DAGScheduler - The Internals of Apache Spark.pdf
DAGScheduler - The Internals of Apache Spark.pdfDAGScheduler - The Internals of Apache Spark.pdf
DAGScheduler - The Internals of Apache Spark.pdfJoeKibangu
 
Improving Apache Spark Downscaling
 Improving Apache Spark Downscaling Improving Apache Spark Downscaling
Improving Apache Spark DownscalingDatabricks
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkPatrick Wendell
 
Debugging & Tuning in Spark
Debugging & Tuning in SparkDebugging & Tuning in Spark
Debugging & Tuning in SparkShiao-An Yuan
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2Fabio Fumarola
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache SparkLucian Neghina
 
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...Inhacking
 
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Аліна Шепшелей
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 

Similar to Apache Spark in Depth: Core Concepts, Architecture & Internals (20)

Data processing platforms with SMACK: Spark and Mesos internals
Data processing platforms with SMACK:  Spark and Mesos internalsData processing platforms with SMACK:  Spark and Mesos internals
Data processing platforms with SMACK: Spark and Mesos internals
 
Spark Deep Dive
Spark Deep DiveSpark Deep Dive
Spark Deep Dive
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache spark - Installation
Apache spark - InstallationApache spark - Installation
Apache spark - Installation
 
Apache spark - Spark's distributed programming model
Apache spark - Spark's distributed programming modelApache spark - Spark's distributed programming model
Apache spark - Spark's distributed programming model
 
Apache Spark II (SparkSQL)
Apache Spark II (SparkSQL)Apache Spark II (SparkSQL)
Apache Spark II (SparkSQL)
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?
 
Core Services behind Spark Job Execution
Core Services behind Spark Job ExecutionCore Services behind Spark Job Execution
Core Services behind Spark Job Execution
 
DAGScheduler - The Internals of Apache Spark.pdf
DAGScheduler - The Internals of Apache Spark.pdfDAGScheduler - The Internals of Apache Spark.pdf
DAGScheduler - The Internals of Apache Spark.pdf
 
Apache Spark Workshop
Apache Spark WorkshopApache Spark Workshop
Apache Spark Workshop
 
Improving Apache Spark Downscaling
 Improving Apache Spark Downscaling Improving Apache Spark Downscaling
Improving Apache Spark Downscaling
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Debugging & Tuning in Spark
Debugging & Tuning in SparkDebugging & Tuning in Spark
Debugging & Tuning in Spark
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache Spark
 
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
 
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 

Recently uploaded

DBA Basics: Getting Started with Performance Tuning.pdf
DBA Basics: Getting Started with Performance Tuning.pdfDBA Basics: Getting Started with Performance Tuning.pdf
DBA Basics: Getting Started with Performance Tuning.pdfJohn Sterrett
 
Dubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls DubaiDubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls Dubaihf8803863
 
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptx
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptxNLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptx
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptxBoston Institute of Analytics
 
Call Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceCall Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceSapana Sha
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfBoston Institute of Analytics
 
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptxEMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptxthyngster
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Jack DiGiovanna
 
How we prevented account sharing with MFA
How we prevented account sharing with MFAHow we prevented account sharing with MFA
How we prevented account sharing with MFAAndrei Kaleshka
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationshipsccctableauusergroup
 
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理e4aez8ss
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxEmmanuel Dauda
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfgstagge
 
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一F La
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024thyngster
 
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAmazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAbdelrhman abooda
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhijennyeacort
 
RABBIT: A CLI tool for identifying bots based on their GitHub events.
RABBIT: A CLI tool for identifying bots based on their GitHub events.RABBIT: A CLI tool for identifying bots based on their GitHub events.
RABBIT: A CLI tool for identifying bots based on their GitHub events.natarajan8993
 
Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Colleen Farrelly
 
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDINTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDRafezzaman
 

Recently uploaded (20)

DBA Basics: Getting Started with Performance Tuning.pdf
DBA Basics: Getting Started with Performance Tuning.pdfDBA Basics: Getting Started with Performance Tuning.pdf
DBA Basics: Getting Started with Performance Tuning.pdf
 
Dubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls DubaiDubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls Dubai
 
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptx
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptxNLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptx
NLP Project PPT: Flipkart Product Reviews through NLP Data Science.pptx
 
Call Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceCall Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts Service
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
 
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptxEMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
 
How we prevented account sharing with MFA
How we prevented account sharing with MFAHow we prevented account sharing with MFA
How we prevented account sharing with MFA
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships
 
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptx
 
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdf
 
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一
办理(UWIC毕业证书)英国卡迪夫城市大学毕业证成绩单原版一比一
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
 
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAmazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
 
RABBIT: A CLI tool for identifying bots based on their GitHub events.
RABBIT: A CLI tool for identifying bots based on their GitHub events.RABBIT: A CLI tool for identifying bots based on their GitHub events.
RABBIT: A CLI tool for identifying bots based on their GitHub events.
 
Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024
 
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDINTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
 

Apache Spark in Depth: Core Concepts, Architecture & Internals

  • 1. Apache Spark in Depth core concepts, architecture & internals Anton Kirillov Ooyala, Mar 2016
  • 2. Roadmap ● RDDs ○ Definition ○ Operations ● Execution workflow ○ DAG ○ Stages and tasks ○ Shuffle ● Architecture ○ Components ○ Memory model ● Coding ○ spark-shell ○ building and submitting Spark applications to YARN
  • 3. Meet Spark ● Generalized framework for distributed data processing (batch, graph, ML) ● Scala collections functional API for manipulating data at scale ● In-memory data caching and reuse across computations ● Applies set of coarse-grained transformations over partitioned data ● Failure recovery relies on lineage to recompute failed tasks ● Supports majority of input formats and integrates with Mesos / YARN
  • 4. Spark makes data engineers happy Backup/restore of Cassandra tables in Parquet def backup(config: Config) { sc.cassandraTable(config.keyspace, config.table).map(_.toEvent).toDF() .write.parquet(config.path) } def restore(config: Config) { sqlContext.read.parquet(config.path) .map(_.toEvent).saveToCassandra(config.keyspace, config.table) } Query different data sources to identify discrepancies sqlContext.sql { """ SELECT count() FROM cassandra_event_rollups JOIN mongo_event_rollups ON cassandra_event_rollups.uuid = cassandra_event_rollups.uuid WHERE cassandra_event_rollups.value != cassandra_event_rollups.value """.stripMargin }
  • 6. RDD: Resilient Distributed Dataset ● A fault-tolerant, immutable, parallel data structure ● Provides API for ○ manipulating the collection of elements (transformations and materialization) ○ persisting intermediate results in memory for later reuse ○ controlling partitioning to optimize data placement ● Can be created through deterministic operation ○ from storage (distributed file system, database, plain file) ○ from another RDD ● Stores information about parent RDDs ○ for execution optimization and operations pipelining ○ to recompute the data in case of failure
  • 7. RDD: a developer’s view ● Distributed immutable data + lazily evaluated operations ○ partitioned data + iterator ○ transformations & actions ● An interface defining 5 main properties a list of partitions (e.g. splits in Hadoop) def getPartitions: Array[Partition] a list of dependencies on other RDDs def getDependencies: Seq[Dependency[_]] a function for computing each split def compute(split: Partition, context: TaskContext): Iterator[T] (optional) a list of preferred locations to compute each split on def getPreferredLocations(split: Partition): Seq[String] = Nil (optional) a partitioner for key-value RDDs val partitioner: Option[Partitioner] = None lineage execution optimization
  • 8. RDDs Example ● HadoopRDD ○ getPartitions = HDFS blocks ○ getDependencies = None ○ compute = load block in memory ○ getPrefferedLocations = HDFS block locations ○ partitioner = None ● MapPartitionsRDD ○ getPartitions = same as parent ○ getDependencies = parent RDD ○ compute = compute parent and apply map() ○ getPrefferedLocations = same as parent ○ partitioner = None sparkContext.textFile("hdfs://...")
  • 9. RDD Operations ● Transformations ○ apply user function to every element in a partition (or to the whole partition) ○ apply aggregation function to the whole dataset (groupBy, sortBy) ○ introduce dependencies between RDDs to form DAG ○ provide functionality for repartitioning (repartition, partitionBy) ● Actions ○ trigger job execution ○ used to materialize computation results ● Extra: persistence ○ explicitly store RDDs in memory, on disk or off-heap (cache, persist) ○ checkpointing for truncating RDD lineage
  • 10. Execution workflow 10 rdd1.join(rdd2) .groupBy(...) .filter(...) splits graph into stages of tasks submits each stage as ready launches tasks via cluster manager retries failed or struggling tasks executes tasks stores and serves blocks
  • 11. Code sample: joining aggregated and raw data //aggregate events after specific date for given campaign val events = sc.cassandraTable("demo", "event") .map(_.toEvent) .filter(event => event.campaignId == campaignId && event.time.isAfter(watermark)) .keyBy(_.eventType) .reduceByKey(_ + _) .cache() //aggregate campaigns by type val campaigns = sc.cassandraTable("demo", "campaign") .map(_.toCampaign) .filter(campaign => campaign.id == campaignId && campaign.time.isBefore(watermark)) .keyBy(_.eventType) .reduceByKey(_ + _) .cache() //joined rollups and raw events val joinedTotals = campaigns.join(events) .map { case (key, (campaign, event)) => CampaignTotals(campaign, event) } .collect() //count totals separately val eventTotals = events.map{ case (t, e) => s"$t -> ${e.value}" }.collect() val campaignTotals = campaigns.map{ case (t, e) => s"$t -> ${e.value}" }.collect()
  • 12. DAG
  • 13. Dependency types ● Narrow (pipelineable) ○ each partition of the parent RDD is used by at most one partition of the child RDD ○ allow for pipelined execution on one cluster node ○ failure recovery is more efficient as only lost parent partitions need to be recomputed ● Wide (shuffle) ○ multiple child partitions may depend on one parent partition ○ require data from all parent partitions to be available and to be shuffled across the nodes ○ if some partition is lost from all the ancestors a complete recomputation is needed
  • 14. Stages and Tasks ● Stages breakdown strategy ○ check backwards from final RDD ○ add each “narrow” dependency to the current stage ○ create new stage when there’s a shuffle dependency ● Tasks ○ ShuffleMapTask partitions its input for shuffle ○ ResultTask sends its output to the driver
  • 15. Shuffle ● Shuffle Write ○ redistributes data among partitions and writes files to disk ○ each hash shuffle task creates one file per “reduce” task (total = MxR) ○ sort shuffle task creates one file with regions assigned to reducer ○ sort shuffle uses in-memory sorting with spillover to disk to get final result ● Shuffle Read ○ fetches the files and applies reduce() logic ○ if data ordering is needed then it is sorted on “reducer” side for any type of shuffle (SPARK-2926)
  • 16. Sort Shuffle ● Incoming records accumulated and sorted in memory according their target partition ids ● Sorted records are written to file or multiple files if spilled and then merged ● index file stores offsets of the data blocks in the data file ● Sorting without deserialization is possible under certain conditions (SPARK-7081)
  • 17. Architecture Recap ● Spark Driver ○ separate process to execute user applications ○ creates SparkContext to schedule jobs execution and negotiate with cluster manager ● Executors ○ run tasks scheduled by driver ○ store computation results in memory, on disk or off-heap ○ interact with storage systems ● Cluster Manager ○ Mesos ○ YARN ○ Spark Standalone
  • 19. Spark Components ● SparkContext ○ represents the connection to a Spark cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster ● DAGScheduler ○ computes a DAG of stages for each job and submits them to TaskScheduler ○ determines preferred locations for tasks (based on cache status or shuffle files locations) and finds minimum schedule to run the jobs ● TaskScheduler ○ responsible for sending tasks to the cluster, running them, retrying if there are failures, and mitigating stragglers ● SchedulerBackend ○ backend interface for scheduling systems that allows plugging in different implementations(Mesos, YARN, Standalone, local) ● BlockManager ○ provides interfaces for putting and retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap)
  • 20. Memory Management in Spark 1.6 ● Execution Memory ○ storage for data needed during tasks execution ○ shuffle-related data ● Storage Memory ○ storage of cached RDDs and broadcast variables ○ possible to borrow from execution memory (spill otherwise) ○ safeguard value is 0.5 of Spark Memory when cached blocks are immune to eviction ● User Memory ○ user data structures and internal metadata in Spark ○ safeguarding against OOM ● Reserved memory ○ memory needed for running executor itself and not strictly related to Spark
  • 21. Workshop code available @ github.com/datastrophic/spark-workshop
  • 22. Execution Modes ● spark-shell --master [ local | spark | yarn-client | mesos] ○ launches REPL connected to specified cluster manager ○ always runs in client mode ● spark-submit --master [ local | spark:// | mesos:// | yarn ] spark-job.jar ○ launches assembly jar on the cluster ● Masters ○ local[k] - run Spark locally with K worker threads ○ spark - launches driver app on Spark Standalone installation ○ mesos - driver will spawn executors on Mesos cluster (deploy-mode: client | cluster) ○ yarn - same idea as with Mesos (deploy-mode: client | cluster) ● Deploy Modes ○ client - driver executed as a separate process on the machine where it has been launched and spawns executors ○ cluster - driver launched as a container using underlying cluster manager
  • 23. Invocation examples spark-shell --master yarn --deploy-mode client --executor-cores 1 --num-executors 2 --jars /target/spark-workshop.jar --conf spark.cassandra.connection.host=cassandra spark-submit --class io.datastrophic.spark.workshop.ParametrizedApplicationExample --master yarn --deploy-mode cluster --num-executors 2 --driver-memory 1g --executor-memory 1g /target/spark-workshop.jar --cassandra-host cassandra --keyspace demo --table event --target-dir /workshop/dumps
  • 24. Live Demo ● spark-shell ● Spark UI ● creating an app with Typesafe Activator ● Spark SQL and DataFrames API ● coding
  • 25. Coding ideas ● get familiar with API through sample project ○ join data from different storage systems ○ aggregate data with breakdown by date ● play with caching and persistence ● check out join behavior applying different partitioning ● familiarize with Spark UI ● experiment with new DataSet API (since 1.6) ● [ your awesome idea here ]