SlideShare a Scribd company logo
1 of 46
Download to read offline
Eric Liang
6/7/2017
TEAM
About Databricks
Started Spark project (now Apache Spark) at UC Berkeley in 2009
22
PRODUCT
Unified Analytics Platform
MISSION
Making Big Data Simple
3
Overview
Deep dive on challenges in writing to Cloud storage (e.g. S3) with
Spark
• Why transactional writes are important
• Cloud storage vs HDFS
• Current solutions are tuned for HDFS
• What we are doing about this
4
What are the requirements for writers?
Spark: unified analytics stack
Common end-to-end use case we've seen: chaining Spark jobs
ETL => analytics
ETL => ETL
Requirements for writes come from downstream Spark jobs
5
Writer example (ETL)
spark.read.csv("s3://source")
.groupBy(...)
.agg(...)
.write.mode("append")
.parquet("s3://dest")
Extract
Transform
Load
6
Reader example (analytics)
spark.read.parquet("s3://dest")
.groupBy(...)
.agg(...)
.show()
Extract
Transform
7
Reader requirement #1
• With chained jobs,
• Failures of upstream jobs shouldn't corrupt data
Distributed
Filesystem
Distributed
Filesystem
read readwrite
Writer job fails
8
Reader requirement #2
• Don't want to read outputs of in-progress jobs
Distributed
Filesystem
Distributed
Filesystem
read
read
read
readwrite
9
Reader requirement #3
• Data shows up (eventual consistency)
read
FileNotFoundException
Cloud
Storage
10
Summary of requirements
Writes should be committed transactionally by Spark
Writes commit atomically
Reads see consistent snapshot
11
How Spark supports this today
• Spark's commit protocol (inherited from Hadoop) ensures
reliable output for jobs
• HadoopMapReduceCommitProtocol:
Spark stages output
files to a temporary
location
Commit?
Move staged files to final
locations
Abort; Delete staged files
yes
no
12
Executor
Executor
Executor
Executor
Driver
Spark commit protocol
Job Start
Task 1
Task 2
Task 3
Task 4 Task 5
Task 6
Commit
Timeline
Task 7
13
Commit on HDFS
• HDFS is the Hadoop distributed filesystem
• Highly available, durable
• Supports fast atomic metadata operations
• e.g. file moves
• HadoopMapReduceCommitProtocol uses series of files
moves for Job commit
• on HDFS, good performance
• close to transactional in practice
14
Does it meet requirements?
Spark commit on HDFS:
1. No partial results on failures => yes*
2. Don't see results of in-progress jobs => yes*
3. Data shows up => yes
* window of failure during commit is small
15
What about the Cloud?
Reliable output works on HDFS, but what about the Cloud?
• Option 1: run HDFS worker on each node (e.g. EC2 instance)
• Option 2: Use Cloud-native storage (e.g. S3)
– Challenge: systems like S3 are not true filesystems
16
Object stores as Filesystems
• Not so hard to provide Filesystem API over object stores such
as S3
• e.g. S3A Filesystem
• Traditional Hadoop applications / Spark continue to work
over Cloud storage using these adapters
•What do you give up (transactionality, performance)?
17
The remainder of this talk
1. Why HDFS has no place in the Cloud
2. Tradeoffs when using existing Hadoop commit protocols with
Cloud storage adapters
3. New commit protocol we built that provides transactional
writes in Databricks
18
Evaluating storage systems
1. Cost
2. SLA (availability and durability)
3. Performance
Let's compare HDFS and S3
19
(1) Cost
• Storage cost per TB-month
• HDFS: $103 with d2.8xl dense storage instances
• S3: $23
• Human cost
• HDFS: team of Hadoop engineers or vendor
• S3: $0
• Elasticity
• Cloud storage is likely >10x cheaper
20
(2) SLA
• Amazon claims 99.99% availability and 99.999999999%
durability.
• Our experience:
• S3 only down twice in last 6 years
• Never lost data
• Most Hadoop clusters have uptime <= 99.9%
21
(3) Performance
• Raw read/write performance
• HDFS offers higher per-node throughput with disk locality
• S3 decouples storage from compute
– performance can scale to your needs
• Metadata performance
• S3: Listing files much slower
– Better w/scalable partition handling in Spark 2.1
• S3: File moves require copies (expensive!)
22
Cloud storage is preferred
• Cloud-native storage wins in cost and SLA
• better price-performance ratio
• more reliable
• However it brings challenges for ETL
• mostly around transactional write requirement
• what is the right commit protocol?
https://databricks.com/blog/2017/05/31/top-5-reasons-for-choos
ing-s3-over-hdfs.html
23
Evaluating commit protocols
• Two commit protocol variations in use today
• HadoopMapReduceCommitProtocol V1
– (as described previously)
• HadoopMapReduceCommitProtocol V2
• DirectOutputCommitter
• deprecated and removed from Spark
• Which one is most suitable for Cloud?
24
Evaluation Criteria
1. Performance – how fast is the protocol at committing files?
2. Transactionality – can a job cause partial or corrupt results to
be visible to readers?
25
Performance Test
Command to write out 100 files to S3:
spark.range(10e6.toLong)
.repartition(100).write.mode("append")
.option(
"mapreduce.fileoutputcommitter.algorithm.version",
version)
.parquet(s"s3:/tmp/test-$version")
26
Performance Test
Time to write out 100 files to S3:
27
Executor
Executor
Executor
Executor
Driver
Hadoop Commit V1
Job Start
Task 1
Task 2
Task 3
Task 4
Task 5
Task 6
Job Commit (moves outputs of tasks 1-6)
Timeline
28
Executor
Executor
Executor
Executor
Driver
Hadoop Commit V2
Job Start
Task 1
Task 2
Task 3
Task 4
Task 5
Task 6
Timeline
Successful task outputs
moved to final locations
ASAP
29
Performance Test
• The V1 commit protocol is slow
• it moves each file serially on the driver
• moves require a copy in S3
• The V2 Hadoop commit protocol is almost five times faster
than V1
• parallel move on executors
30
Transactionality
• Example: What happens if a job fails due to a bad task?
• Common situation if certain input files have bad records
• The job should fail cleanly
• No partial outputs visible to readers
• Let's evaluate Hadoop commit protocols {V1, V2} again
31
Fault Test
Simulate a job failure due to bad records:
spark.range(10000).repartition(7).map { i =>
if (i == 123) { throw new RuntimeException("oops!") }
else i
}.write.option("mapreduce.fileoutputcommitter.algorithm.version
", version)
.mode("append").parquet(s"s3:/tmp/test-$version")
32
Fault Test
Did the failed job leave behind any partial results?
33
Fault Test
• The V2 protocol left behind ~8k garbage rows for the failed
job.
• This is duplicate data that needs to be "manually" cleaned up before
running the job again
• We see empirically that while V2 is faster, it also leaves behind
partial results on job failures, breaking transactionality
requirements
34
Concurrent Readers Test
• Wait, is V1 really transactional?
spark.range(1e9.toLong)
.repartition(70)
.write.mode("append")
.parquet("s3:/test")
spark.read("s3:/test").count()
> 0
> 0
> 14285714
> 814285714
> 1000000000
35
Transactionality Test
• V1 isn't very "transactional" on Cloud storage either
• Long commit phase
• What if driver fails?
• What if another job reads concurrently?
• On HDFS the window of vulnerability is much smaller because
file moves are O(millisecond)
36
The problem with Hadoop commit
• Key design assumption of Hadoop commit V1: file moves are
fast
• V2 is a workaround for Cloud storage, but sacrifices
transactionality
• This tradeoff isn't fundamental
• How can we get both strong transactionality guarantees and
the best performance?
37
Executor
Executor
Executor
Executor
Driver
Ideal commit protocol?
Job Start
Task 1
Task 2
Task 3
Task 4
Task 5
Task 6
Timeline
Instantaneous commit
38
No compromises with DBIO
transactional commit
• When a user writes a file in a job
• Embed a unique txn id in the file name
• Write the file directly to its final location
• Mark the txn id as committed if the job
commits
• When a user is listing files
• Ignore the file if it has a txn id that is
uncommitted
DBIO
Cloud
Storage
39
DBIO transactional commit
40
Fault analysis
41
Concurrent Readers Test
• DBIO provides atomic commit
spark.range(1e9.toLong)
.repartition(70)
.write.mode("append")
.parquet("s3:/test")
spark.read("s3:/test").count()
> 0
> 0
> 0
> 0
> 1000000000
42
Other benefits of DBIO commit
• High performance
• Strong correctness guarantees
+ Safe Spark task speculation
+ Atomically add and remove sets of files
+ Enhanced consistency
43
Implementation challenges
• Challenge: compatibility with external systems like Hive
• Solution:
• relaxed transactionality guarantees: If you read from an external
system you might read uncommitted files as before
• %sql VACUUM command to clean up files (also auto-vacuum)
• Challenge: how to reliably track transaction status
• Solution:
• hybrid of metadata files and service component
• more data warehousing features in the future
44
Rollout status
Enabled by default starting with Spark 2.1-db5 in Databricks
Process of gradual rollout
>100 customers already using DBIO transactional commit
45
Summary
• Cloud Storage >> HDFS, however
• Different design required for commit protocols
• No compromises with DBIO transactional commit
See our engineering blog post for more details
https://databricks.com/blog/2017/05/31/transactional-writes-clo
ud-storage.html
UNIFIED ANALYTICS PLATFORM
Try Apache Spark in Databricks!
• Collaborative cloud environment
• Free version (community edition)
4646
DATABRICKS RUNTIME 3.0
• Apache Spark - optimized for the cloud
• Caching and optimization layer - DBIO
• Enterprise security - DBES
Try for free today.
databricks.com

More Related Content

What's hot

Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsDatabricks
 
How to Automate Performance Tuning for Apache Spark
How to Automate Performance Tuning for Apache SparkHow to Automate Performance Tuning for Apache Spark
How to Automate Performance Tuning for Apache SparkDatabricks
 
Dynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache SparkDynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache SparkDatabricks
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsDatabricks
 
Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIDatabricks
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsDatabricks
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkRahul Jain
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming JobsDatabricks
 
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
 
Moving to Databricks & Delta
Moving to Databricks & DeltaMoving to Databricks & Delta
Moving to Databricks & DeltaDatabricks
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Databricks
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLDatabricks
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark Aakashdata
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudDatabricks
 
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
 
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)

Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
 
Spark SQL
Spark SQLSpark SQL
Spark SQL
 
How to Automate Performance Tuning for Apache Spark
How to Automate Performance Tuning for Apache SparkHow to Automate Performance Tuning for Apache Spark
How to Automate Performance Tuning for Apache Spark
 
Dynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache SparkDynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache Spark
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark API
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
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
 
Moving to Databricks & Delta
Moving to Databricks & DeltaMoving to Databricks & Delta
Moving to Databricks & Delta
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the Cloud
 
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
 
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...
 

Similar to Transactional writes to cloud storage with Eric Liang

Robust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkRobust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkDatabricks
 
A Journey into Databricks' Pipelines: Journey and Lessons Learned
A Journey into Databricks' Pipelines: Journey and Lessons LearnedA Journey into Databricks' Pipelines: Journey and Lessons Learned
A Journey into Databricks' Pipelines: Journey and Lessons LearnedDatabricks
 
[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼NAVER D2
 
5 Pitfalls to Avoid with MongoDB
5 Pitfalls to Avoid with MongoDB5 Pitfalls to Avoid with MongoDB
5 Pitfalls to Avoid with MongoDBTim Callaghan
 
Spark to DocumentDB connector
Spark to DocumentDB connectorSpark to DocumentDB connector
Spark to DocumentDB connectorDenny Lee
 
Docker interview Questions-3.pdf
Docker interview Questions-3.pdfDocker interview Questions-3.pdf
Docker interview Questions-3.pdfYogeshwaran R
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoopclairvoyantllc
 
Teaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud PlatformTeaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud PlatformYao Yao
 
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...Amazon Web Services Japan
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupRafal Kwasny
 
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Cloudera, Inc.
 
Supporting Digital Media Workflows in the Cloud with Perforce Helix
Supporting Digital Media Workflows in the Cloud with Perforce HelixSupporting Digital Media Workflows in the Cloud with Perforce Helix
Supporting Digital Media Workflows in the Cloud with Perforce HelixPerforce
 
Optimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public CloudOptimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public CloudQubole
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Djamel Zouaoui
 
Building FoundationDB
Building FoundationDBBuilding FoundationDB
Building FoundationDBFoundationDB
 
Scaling spark on kubernetes at Lyft
Scaling spark on kubernetes at LyftScaling spark on kubernetes at Lyft
Scaling spark on kubernetes at LyftLi Gao
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark TutorialAhmet Bulut
 
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks Delta
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks DeltaEnd-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks Delta
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks DeltaDatabricks
 

Similar to Transactional writes to cloud storage with Eric Liang (20)

Robust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkRobust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache Spark
 
[AWS Builders] Effective AWS Glue
[AWS Builders] Effective AWS Glue[AWS Builders] Effective AWS Glue
[AWS Builders] Effective AWS Glue
 
A Journey into Databricks' Pipelines: Journey and Lessons Learned
A Journey into Databricks' Pipelines: Journey and Lessons LearnedA Journey into Databricks' Pipelines: Journey and Lessons Learned
A Journey into Databricks' Pipelines: Journey and Lessons Learned
 
[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼
 
5 Pitfalls to Avoid with MongoDB
5 Pitfalls to Avoid with MongoDB5 Pitfalls to Avoid with MongoDB
5 Pitfalls to Avoid with MongoDB
 
Spark to DocumentDB connector
Spark to DocumentDB connectorSpark to DocumentDB connector
Spark to DocumentDB connector
 
Docker interview Questions-3.pdf
Docker interview Questions-3.pdfDocker interview Questions-3.pdf
Docker interview Questions-3.pdf
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoop
 
Teaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud PlatformTeaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud Platform
 
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...
202201 AWS Black Belt Online Seminar Apache Spark Performnace Tuning for AWS ...
 
Typesafe spark- Zalando meetup
Typesafe spark- Zalando meetupTypesafe spark- Zalando meetup
Typesafe spark- Zalando meetup
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
 
Supporting Digital Media Workflows in the Cloud with Perforce Helix
Supporting Digital Media Workflows in the Cloud with Perforce HelixSupporting Digital Media Workflows in the Cloud with Perforce Helix
Supporting Digital Media Workflows in the Cloud with Perforce Helix
 
Optimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public CloudOptimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public Cloud
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming
 
Building FoundationDB
Building FoundationDBBuilding FoundationDB
Building FoundationDB
 
Scaling spark on kubernetes at Lyft
Scaling spark on kubernetes at LyftScaling spark on kubernetes at Lyft
Scaling spark on kubernetes at Lyft
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks Delta
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks DeltaEnd-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks Delta
End-to-End Spark/TensorFlow/PyTorch Pipelines with Databricks Delta
 

More from Databricks

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDatabricks
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Databricks
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Databricks
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Databricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of HadoopDatabricks
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDatabricks
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceDatabricks
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringDatabricks
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixDatabricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationDatabricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchDatabricks
 
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
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesDatabricks
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsDatabricks
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkDatabricks
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkDatabricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesDatabricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkDatabricks
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeDatabricks
 

More from Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
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
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 

Recently uploaded

Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 217djon017
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data VisualizationKianJazayeri1
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksdeepakthakur548787
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our WorldEduminds Learning
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Seán Kennedy
 
Advanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsAdvanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsVICTOR MAESTRE RAMIREZ
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Boston Institute of Analytics
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...Dr Arash Najmaei ( Phd., MBA, BSc)
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Thomas Poetter
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Cathrine Wilhelmsen
 
Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Seán Kennedy
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Boston Institute of Analytics
 
convolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfconvolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfSubhamKumar3239
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesTimothy Spann
 
Cyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataCyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataTecnoIncentive
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxaleedritatuxx
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxTasha Penwell
 
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...KarteekMane1
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxSimranPal17
 

Recently uploaded (20)

Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data Visualization
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing works
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our World
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...
 
Advanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsAdvanced Machine Learning for Business Professionals
Advanced Machine Learning for Business Professionals
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)
 
Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
 
convolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfconvolutional neural network and its applications.pdf
convolutional neural network and its applications.pdf
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
 
Cyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataCyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded data
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
 
Data Analysis Project: Stroke Prediction
Data Analysis Project: Stroke PredictionData Analysis Project: Stroke Prediction
Data Analysis Project: Stroke Prediction
 
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptx
 

Transactional writes to cloud storage with Eric Liang

  • 2. TEAM About Databricks Started Spark project (now Apache Spark) at UC Berkeley in 2009 22 PRODUCT Unified Analytics Platform MISSION Making Big Data Simple
  • 3. 3 Overview Deep dive on challenges in writing to Cloud storage (e.g. S3) with Spark • Why transactional writes are important • Cloud storage vs HDFS • Current solutions are tuned for HDFS • What we are doing about this
  • 4. 4 What are the requirements for writers? Spark: unified analytics stack Common end-to-end use case we've seen: chaining Spark jobs ETL => analytics ETL => ETL Requirements for writes come from downstream Spark jobs
  • 7. 7 Reader requirement #1 • With chained jobs, • Failures of upstream jobs shouldn't corrupt data Distributed Filesystem Distributed Filesystem read readwrite Writer job fails
  • 8. 8 Reader requirement #2 • Don't want to read outputs of in-progress jobs Distributed Filesystem Distributed Filesystem read read read readwrite
  • 9. 9 Reader requirement #3 • Data shows up (eventual consistency) read FileNotFoundException Cloud Storage
  • 10. 10 Summary of requirements Writes should be committed transactionally by Spark Writes commit atomically Reads see consistent snapshot
  • 11. 11 How Spark supports this today • Spark's commit protocol (inherited from Hadoop) ensures reliable output for jobs • HadoopMapReduceCommitProtocol: Spark stages output files to a temporary location Commit? Move staged files to final locations Abort; Delete staged files yes no
  • 12. 12 Executor Executor Executor Executor Driver Spark commit protocol Job Start Task 1 Task 2 Task 3 Task 4 Task 5 Task 6 Commit Timeline Task 7
  • 13. 13 Commit on HDFS • HDFS is the Hadoop distributed filesystem • Highly available, durable • Supports fast atomic metadata operations • e.g. file moves • HadoopMapReduceCommitProtocol uses series of files moves for Job commit • on HDFS, good performance • close to transactional in practice
  • 14. 14 Does it meet requirements? Spark commit on HDFS: 1. No partial results on failures => yes* 2. Don't see results of in-progress jobs => yes* 3. Data shows up => yes * window of failure during commit is small
  • 15. 15 What about the Cloud? Reliable output works on HDFS, but what about the Cloud? • Option 1: run HDFS worker on each node (e.g. EC2 instance) • Option 2: Use Cloud-native storage (e.g. S3) – Challenge: systems like S3 are not true filesystems
  • 16. 16 Object stores as Filesystems • Not so hard to provide Filesystem API over object stores such as S3 • e.g. S3A Filesystem • Traditional Hadoop applications / Spark continue to work over Cloud storage using these adapters •What do you give up (transactionality, performance)?
  • 17. 17 The remainder of this talk 1. Why HDFS has no place in the Cloud 2. Tradeoffs when using existing Hadoop commit protocols with Cloud storage adapters 3. New commit protocol we built that provides transactional writes in Databricks
  • 18. 18 Evaluating storage systems 1. Cost 2. SLA (availability and durability) 3. Performance Let's compare HDFS and S3
  • 19. 19 (1) Cost • Storage cost per TB-month • HDFS: $103 with d2.8xl dense storage instances • S3: $23 • Human cost • HDFS: team of Hadoop engineers or vendor • S3: $0 • Elasticity • Cloud storage is likely >10x cheaper
  • 20. 20 (2) SLA • Amazon claims 99.99% availability and 99.999999999% durability. • Our experience: • S3 only down twice in last 6 years • Never lost data • Most Hadoop clusters have uptime <= 99.9%
  • 21. 21 (3) Performance • Raw read/write performance • HDFS offers higher per-node throughput with disk locality • S3 decouples storage from compute – performance can scale to your needs • Metadata performance • S3: Listing files much slower – Better w/scalable partition handling in Spark 2.1 • S3: File moves require copies (expensive!)
  • 22. 22 Cloud storage is preferred • Cloud-native storage wins in cost and SLA • better price-performance ratio • more reliable • However it brings challenges for ETL • mostly around transactional write requirement • what is the right commit protocol? https://databricks.com/blog/2017/05/31/top-5-reasons-for-choos ing-s3-over-hdfs.html
  • 23. 23 Evaluating commit protocols • Two commit protocol variations in use today • HadoopMapReduceCommitProtocol V1 – (as described previously) • HadoopMapReduceCommitProtocol V2 • DirectOutputCommitter • deprecated and removed from Spark • Which one is most suitable for Cloud?
  • 24. 24 Evaluation Criteria 1. Performance – how fast is the protocol at committing files? 2. Transactionality – can a job cause partial or corrupt results to be visible to readers?
  • 25. 25 Performance Test Command to write out 100 files to S3: spark.range(10e6.toLong) .repartition(100).write.mode("append") .option( "mapreduce.fileoutputcommitter.algorithm.version", version) .parquet(s"s3:/tmp/test-$version")
  • 26. 26 Performance Test Time to write out 100 files to S3:
  • 27. 27 Executor Executor Executor Executor Driver Hadoop Commit V1 Job Start Task 1 Task 2 Task 3 Task 4 Task 5 Task 6 Job Commit (moves outputs of tasks 1-6) Timeline
  • 28. 28 Executor Executor Executor Executor Driver Hadoop Commit V2 Job Start Task 1 Task 2 Task 3 Task 4 Task 5 Task 6 Timeline Successful task outputs moved to final locations ASAP
  • 29. 29 Performance Test • The V1 commit protocol is slow • it moves each file serially on the driver • moves require a copy in S3 • The V2 Hadoop commit protocol is almost five times faster than V1 • parallel move on executors
  • 30. 30 Transactionality • Example: What happens if a job fails due to a bad task? • Common situation if certain input files have bad records • The job should fail cleanly • No partial outputs visible to readers • Let's evaluate Hadoop commit protocols {V1, V2} again
  • 31. 31 Fault Test Simulate a job failure due to bad records: spark.range(10000).repartition(7).map { i => if (i == 123) { throw new RuntimeException("oops!") } else i }.write.option("mapreduce.fileoutputcommitter.algorithm.version ", version) .mode("append").parquet(s"s3:/tmp/test-$version")
  • 32. 32 Fault Test Did the failed job leave behind any partial results?
  • 33. 33 Fault Test • The V2 protocol left behind ~8k garbage rows for the failed job. • This is duplicate data that needs to be "manually" cleaned up before running the job again • We see empirically that while V2 is faster, it also leaves behind partial results on job failures, breaking transactionality requirements
  • 34. 34 Concurrent Readers Test • Wait, is V1 really transactional? spark.range(1e9.toLong) .repartition(70) .write.mode("append") .parquet("s3:/test") spark.read("s3:/test").count() > 0 > 0 > 14285714 > 814285714 > 1000000000
  • 35. 35 Transactionality Test • V1 isn't very "transactional" on Cloud storage either • Long commit phase • What if driver fails? • What if another job reads concurrently? • On HDFS the window of vulnerability is much smaller because file moves are O(millisecond)
  • 36. 36 The problem with Hadoop commit • Key design assumption of Hadoop commit V1: file moves are fast • V2 is a workaround for Cloud storage, but sacrifices transactionality • This tradeoff isn't fundamental • How can we get both strong transactionality guarantees and the best performance?
  • 37. 37 Executor Executor Executor Executor Driver Ideal commit protocol? Job Start Task 1 Task 2 Task 3 Task 4 Task 5 Task 6 Timeline Instantaneous commit
  • 38. 38 No compromises with DBIO transactional commit • When a user writes a file in a job • Embed a unique txn id in the file name • Write the file directly to its final location • Mark the txn id as committed if the job commits • When a user is listing files • Ignore the file if it has a txn id that is uncommitted DBIO Cloud Storage
  • 41. 41 Concurrent Readers Test • DBIO provides atomic commit spark.range(1e9.toLong) .repartition(70) .write.mode("append") .parquet("s3:/test") spark.read("s3:/test").count() > 0 > 0 > 0 > 0 > 1000000000
  • 42. 42 Other benefits of DBIO commit • High performance • Strong correctness guarantees + Safe Spark task speculation + Atomically add and remove sets of files + Enhanced consistency
  • 43. 43 Implementation challenges • Challenge: compatibility with external systems like Hive • Solution: • relaxed transactionality guarantees: If you read from an external system you might read uncommitted files as before • %sql VACUUM command to clean up files (also auto-vacuum) • Challenge: how to reliably track transaction status • Solution: • hybrid of metadata files and service component • more data warehousing features in the future
  • 44. 44 Rollout status Enabled by default starting with Spark 2.1-db5 in Databricks Process of gradual rollout >100 customers already using DBIO transactional commit
  • 45. 45 Summary • Cloud Storage >> HDFS, however • Different design required for commit protocols • No compromises with DBIO transactional commit See our engineering blog post for more details https://databricks.com/blog/2017/05/31/transactional-writes-clo ud-storage.html
  • 46. UNIFIED ANALYTICS PLATFORM Try Apache Spark in Databricks! • Collaborative cloud environment • Free version (community edition) 4646 DATABRICKS RUNTIME 3.0 • Apache Spark - optimized for the cloud • Caching and optimization layer - DBIO • Enterprise security - DBES Try for free today. databricks.com