SlideShare una empresa de Scribd logo
1 de 43
Uncovering an Apache Spark 2 Benchmark -
Configuration, Tuning and Test Results
• Mark Lochbihler, Hortonworks - Principal Architect
• Viplava Madasu, HPE - Big Data Systems Engineer
San Jose, California
JUNE 17–21, 2018
1
Tuesday, June 19
4:00 PM - 4:40 PM
Executive Ballroom
210C/D/G/H
Today’s Agenda
• What’s New with Spark 2.x – Mark
• Spark Architecture
• Spark on YARN
• What’s New
• Spark 2.x Benchmark - Viplava
• What was Benchmarked
• Configuration and Tuning
• Infrastructure Used
• Results
• Questions / More Info – Mark and Viplava
San Jose, California
JUNE 17–21, 20182
Apache Spark
Apache Spark is a fast general-purpose engine for large-scale data
processing. Spark was developed in response to limitations in Hadoop’s
two-stage disk-based MapReduce processing framework.
Orchestration:
Spark’s standalone cluster manager, Apache Mesos,
or Hadoop YARN San Jose, California
JUNE 17–21, 2018
3
Spark on Hadoop YARN
YARN has the concept of labels for groupings of Hadoop Worker nodes.
Spark on YARN is an optimal way to schedule and run Spark jobs on a Hadoop cluster alongside a variety of
other data-processing frameworks, leveraging existing clusters using queue placement policies, and enabling
security by running on Kerberos-enabled clusters.
Client Mode Cluster Mode
Client
Executor
App
MasterSpark Driver
Client
Executor
App Master
Spark Driver
San Jose, California
JUNE 17–21, 20184
Spark 2.x vs Spark 1.x
Apache Spark 2.x is a major release update of Spark 1.x and includes
significant updates in the following areas:
• API usability
• SQL 2003 support
• Performance improvements
• Structured streaming
• R UDF support
• Operational improvements
San Jose, California
JUNE 17–21, 2018
5
Spark 2.x – New and Updated APIs
Including:
• Unifying DataFrame and Dataset APIs providing type safety for
DataFrames
• New SparkSession API with a new entry point that replaces the old
SQLContext and HiveContext for DataFrame and Dataset APIs
• New streamlined configuration API for SparkSession
• New improved Aggregator API for typed aggregation in Datasets
San Jose, California
JUNE 17–21, 2018
6
Spark 2.x – Improved SQL Functionality
• ANSI SQL 2003 support
• Enables running all 99 TPC-DS queries
• A native SQL parser that supports both ANSI-SQL as well as Hive QL
• Native DDL command implementations
• Subquery support
• Native CSV data source
• Off-heap memory management for both caching and runtime
execution
• Hive-style bucketing support
San Jose, California
JUNE 17–21, 2018
7
Spark 2.x – Performance Improvements
• By implementing a new technique called “whole stage code
generation”, Spark 2.x improves the performance 2-10 times for
common operators in SQL and DataFrames.
• Other performance improvements include:
• Improved Parquet scan throughput through vectorization
• Improved ORC performance
• Many improvements in the Catalyst query optimizer for common workloads
• Improved window function performance via native implementations
for all window functions.
San Jose, California
JUNE 17–21, 2018
8
Spark 2.x – Spark Machine Learning API
• Spark 2.x replaces the RDD-based APIs in the spark.mllib package (put in
maintenance mode) with the DataFrame-based API in the spark.ml
package.
• New features in the Spark 2.x Machine Learning API include:
• ML persistence to support saving and loading ML models and Pipelines
• New MLlib APIs in R for generalized linear models
• Naive Bayes
• K-Means Clustering
• Survival regression
• New MLlib APIs in Python for
• LDA, Gaussian Mixture Model, Generalized Linear Regression, etc.
San Jose, California
JUNE 17–21, 2018
9
Spark 2.x – Spark Streaming
• Spark 2.x introduced a new high-level streaming API, called
Structured Streaming, built on top of Spark SQL and the Catalyst
optimizer.
• Structured Streaming enables users to program against streaming
sources and sinks using the same DataFrame/Dataset API as in static
data sources, leveraging the Catalyst optimizer to automatically
incrementalize the query plans.
San Jose, California
JUNE 17–21, 2018
10
11
Hortonworks Data Platform 2.6.5 – Just Released
HDP 2.6.5 / 3.0 includes Apache Spark 2.3
ORC/Parquet Feature Parity
– Spark extends its vectorized read capability to ORC data sources.
– Structured streaming officially supports ORC data source with API and documentation
Python Pandas UDF, with good performance and easy to use for Pandas users. This feature supports
financial analysis use cases.
Structured streaming now supports stream-stream joins.
Structured streaming that goes to millisecond latency (Alpha). New continuous processing mode
provides the best performance by minimizing the latency without waiting in idle status.
San Jose, California
JUNE 17–21, 2018
Evaluation of Spark SQL with Spark 2.x versus Spark 1.6
• Benchmark Performed
• Hive testbench, which is similar to TPC-DS benchmark
• Tuning for the benchmark
San Jose, California
JUNE 17–21, 2018
12
Why Cluster tuning matters
• Spark/Hadoop default configurations are not optimal for most enterprise
applications
• Large number of configuration parameters
• Tuning cluster will benefit all the applications
• Can further tune job level configuration
• More important if using disaggregated compute/storage layers as in HPE
Reference Architecture
• Useful for cloud too
San Jose, California
JUNE 17–21, 2018
13
Factors to consider for Spark performance tuning
• Hardware
• CPU, Memory, Storage systems, Local disks, Network
• Hadoop configuration
• HDFS
• YARN
• Spark configuration
• Executor cores, Executor memory, Shuffle partitions, Compression etc.
San Jose, California
JUNE 17–21, 2018
14
General Hardware Guidelines
• Sizing hardware for Spark depends on the use case, but Spark benefits from
• More CPU cores
• More memory
• Flash storage for temporary storage
• Faster network fabric
• CPU Cores
• Spark scales well to tens of CPU cores per machine
• Most Spark applications are CPU bound, so at least 8-16 cores per machine.
• Memory
• Spark can make use of hundreds of gigabytes of memory per machine
• Allocate only at most 75% of the memory for Spark; leave the rest for the operating
system and buffer cache.
• Storage tab of Spark’s monitoring UI will help.
• Max 200GB per executor.
San Jose, California
JUNE 17–21, 2018
15
General Hardware Guidelines …
• Network
• For Group-By, Reduce-By, and SQL join operations, network performance
becomes important due to the Shuffles involved
• 10 Gigabit network is the recommended choice
• Local Disks
• Spark uses local disks to store data that doesn’t fit in RAM, as well as to preserve
intermediate output between stages
• SSDs are recommended
• Mount disks with noatime option to reduce unnecessary writes
San Jose, California
JUNE 17–21, 2018
16
Spark on Hadoop/YARN
(Cluster Mode)
San Jose, California
JUNE 17–21, 2018
17
Useful HDFS configuration settings
• Increase the dfs.blocksize value to allow more data to be processed by
each map task
• Also reduces NameNode memory consumption
• dfs.blocksize 256/512MB
• Increase the dfs.namenode.handler.count value to better manage
multiple HDFS operations from multiple clients
• dfs.namenode.handler.count 100
• To eliminate timeout exceptions (java.io.IOException: Unable to close file
close file because the last block does not have enough number of replicas),
San Jose, California
JUNE 17–21, 2018
18
Useful YARN configuration settings
• YARN is the popular cluster manager for Spark on Hadoop, so it is
important that YARN and Spark configurations are tuned in tandem.
• Settings of Spark executor memory and executor cores result in
allocation requests to YARN with the same values and YARN should be
configured to accommodate the desired Spark settings
• Amount of physical memory that can be allocated for containers per
node
• yarn.nodemanager.resource.memory-mb 384 GiB
• Amount of vcores available on a compute node that can be allocated for
containers
• yarn.nodemanager.resource.cpu-vcores 48
San Jose, California
JUNE 17–21, 2018
19
YARN tuning …
• Number of YARN containers depends on the nature of the workload
• Assuming total of 384 GiB on each node, a workload that needs 24 GiB containers
will result in 16 total containers
• Assuming 12 worker nodes, number of 24 GiB containers = 16 * 12 – 1 = 191
• One container per YARN application master
• General guideline is to configure containers in a way that maximizes the
utilization of the memory and vcores on each node in the cluster
San Jose, California
JUNE 17–21, 2018
20
YARN tuning …
• Location of YARN intermediate files on the compute nodes
• yarn.nodemanager.local-dirs /data1/hadoop/yarn/local, /data2/hadoop/yarn/local,
/data3/hadoop/yarn/local, /data4/hadoop/yarn/local
• Setting of spark.local.dir is ignored for YARN cluster mode
• The node-locality-delay specifies how many scheduling intervals to let
pass attempting to find a node local slot to run on prior to searching for a
rack local slot
• Important for small jobs that do not have a large number of tasks as it will better
utilize the compute nodes
• yarn.scheduler.capacity.node-locality-delay 1
San Jose, California
JUNE 17–21, 2018
21
Tuning Spark – Executor cores
• Unlike Hadoop MapReduce where each map or reduce task is always started in a new
process, Spark can efficiently use process threads (cores) to distribute task processing
• Results in a need to tune Spark executors with respect to the amount of memory
and number of cores each executor can use
• Has to work within the configuration boundaries of YARN
• Number of cores per executor can be controlled by
• the configuration setting spark.executor.cores
• the --executor-cores option of the spark-submit command
• The default is 1 for Spark on YARN
San Jose, California
JUNE 17–21, 2018
22
Tuning Spark – Executor cores
• Simplest but inefficient approach would be to configure one executor per core and divide the memory
equally among the number of executors
• Since each partition cannot be computed on more than one executor, the size of each partition is
limited and causes memory problems, or spilling to disk for shuffles
• If the executors have only one core, then at most one task can run in each executor, which throws
away the benefits of broadcast variables, which have to be sent to each executor once.
• Each executor has some memory overhead (minimum of 384MB) – so, if we have many small
executors, results in lot of memory overhead
• Giving many cores to each executor also has issues
• GC issues - since a larger JVM heap will delay the time until a GC event is triggered resulting in
larger GC pauses
• Results in poor HDSF throughput issues because of handling many concurrent threads
• spark.executor.cores – experiment and set this based on your workloads. We found 9 was
was the right setting for this configuration and bench test in our lab.
San Jose, California
JUNE 17–21, 2018
23
Tuning Spark – Memory
• Memory for each Spark job is application specific
• Configure Executor memory in proportion to the number of partitions and cores per
executor
• Divide the total amount of memory on each node by the number of executors on the node
• Should be less than the maximum YARN container size - so YARN maximum container size may
need to be adjusted accordingly
• Configuration setting spark.executor.memory or the --executor-memory option of the spark-
submit command
• JVM runs into issues with very large heaps (above 80GB).
• Spark Driver memory
• If driver collects too much data, the job may run into OOM errors.
• Increase the driver memory using spark.driver.maxResultSize
San Jose, California
JUNE 17–21, 2018
24
Spark 2.x – Memory Model
• Each executor has memory overhead for things like VM
overheads, interned strings, other native overheads
• spark.yarn.executor.memoryOverhead
• Default value is spark.executor.memory * 0.10, with minimum of
384MB.
• Prior to Spark 1.6, separate tuning was needed for
storage (RDD) memory and execution/shuffle memory
via spark.storage.memoryFraction and
spark.shuffle.memoryFraction
• Spark 1.6 introduced a new “UnifiedMemoryManager”
• When no Storage memory is used, Execution can acquire all the
available memory and vice versa
• As a result, applications that do not use caching can use the
entire space for execution, obviating unnecessary disk spills.
• Applications that do use caching can reserve a minimum storage
space where their data blocks are immune to being evicted
• spark.memory.storageFraction tunable, but good out-of-the-box
performance
San Jose, California
JUNE 17–21, 2018
25
Tuning Spark – Shuffle partitions
• Spark SQL, by default, sets the number of reduce side
partitions to 200 when doing a shuffle for wide
transformations, e.g., groupByKey, reduceByKey,
sortByKey etc.
• Not optimal for many cases as it will use only 200 cores for
processing tasks after the shuffle
• For large datasets, this might result in shuffle block overflow
resulting in job failures
• The number of shuffle partitions should be at least
equal to the number of total executor cores or a
multiple of it in case of large data sets.
• spark.sql.shuffle.partitions setting
• Also – helps to partition in prime numbers in terms of
hash effectiveness.
San Jose, California
JUNE 17–21, 2018
26
Tuning Spark – Compression
• Using compression in Spark can improve performance in a meaningful
way as compression results in less disk I/O and network I/O
• Even though compressing the data results in some CPU cycles being
used, the performance improvements with compression outweigh the
CPU overhead when a large amount of data is involved
• Also compression results in reduced storage requirements for storing
data on disk, e.g., intermediate shuffle files
San Jose, California
JUNE 17–21, 2018
27
Tuning Spark – Compression
• spark.io.compression.codec setting to decide the codec
• three codecs provided: lz4, lzf, and snappy
• default codec is lz4
• Four main places where Spark makes use of compression
• Compress map output files during a shuffle operation using
spark.shuffle.compress setting (Default true)
• Compress data spilled during shuffles using spark.shuffle.spill.compress setting
(Default true)
• Compress broadcast variables before sending them using
spark.broadcast.compress setting (Default true)
• Compress serialized RDD partitions using spark.rdd.compress setting
(Default false)
San Jose, California
JUNE 17–21, 2018
28
Tuning Spark – Serialization type
• Serialization plays an important role in the performance of any distributed application
• Spark memory usage is greatly affected by storage level and serialization format
• By default, Spark serializes objects using Java Serializer which can work with any class that implements
java.io.Serializable interface
• For custom data types, Kryo Serialization is more compact and efficient than Java Serialization
• but user classes need to be explicitly registered with the Kryo Serializer
• spark.serializer org.apache.spark.serializer.KryoSerializer
• Spark SQL automatically uses Kryo serialization for DataFrames internally in Spark 2.x
• For customer applications that still use RDDs, Kryo Serialization should result in a significant
performance boost
San Jose, California
JUNE 17–21, 2018
29
Tuning Spark – Other configuration settings
• When using ORC/parquet format for the data, Spark SQL can push the filter
down to ORC/parquet, thus avoiding large data transfer.
• spark.sql.orc.filterPushdown (Default false)
• spark.sql.parquet.filterPushdown (Default true)
• For large data sets, you may encounter various network timeouts. Can tune
different timeout values
• spark.core.connection.ack.wait.timeout
• spark.storage.blockManagerSlaveTimeoutMs
• spark.shuffle.io.connectionTimeout
• spark.rpc.askTimeout
• spark.rpc.lookupTimeout
• “Umbrella” setting for all these timeouts, spark.network.timeout (Default is
120 seconds). For 10TB dataset, this value should be something like 600
seconds.
San Jose, California
JUNE 17–21, 2018
30
HPE’s Elastic Platform for Big Data Analytics (EPA)
Modular building blocks of compute and storage optimized for modern workloads
Apollo 2000
Compute
DL360 Apollo 6500
w/ NVIDIA GPU
Synergy
Storage
Apollo 4200DL380 Apollo 4510
Hot Cold Object
Purpose - built
Network FlexFabric 5950/5940
San Jose, California
JUNE 17–21, 201831
HPE EPA - Single-Rack Reference Architecture for Spark 2.x
San Jose, California
JUNE 17–21, 201832
HPE EPA - Single-Rack Reference Architecture for Spark 2.x
San Jose, California
JUNE 17–21, 201833
San Jose, California
JUNE 17–21, 2018
Base Rack
• (1) DL360 Control Block – (1)
Management Node, (2) Head Nodes
• (8) Apollo 2000 Compute Blocks –
(32) XL170r Worker Nodes
• (10) Apollo 4200 Storage Blocks –
(10) Apollo 4200 Data Nodes
• (1) Network Block
• (1) Rack Block
Aggregation Rack
• (8) Apollo 2000 Compute Blocks – (32)
XL170r Worker Nodes
• (10) Apollo 4200 Storage Blocks – (10)
Apollo 4200 Data Nodes
• (1) Network Block
• (1) Aggregation Switch Block - (2) HPE
5950 32QSFP28
• (1) Rack Block
Expansion Rack
• (8) Apollo 2000 Compute Blocks –
(32) XL170r Worker Nodes
• (10) Apollo 4200 Storage Blocks –
(10) Apollo 4200 Data Nodes
• (1) Network Block
• (1) Rack Block
Expansion Rack
• (8) Apollo 2000 Compute Blocks –
(32) XL170r Worker Nodes
• (10) Apollo 4200 Storage Blocks –
(10) Apollo 4200 Data Nodes
• (1) Network Block
• (1) Rack Block
HPE EPA - Multi-Rack configuration
34
Spark 2.x - Effect of cores per executor on query performance
San Jose, California
JUNE 17–21, 2018
35
Spark 2.x – Effect of shuffle partitions on query performance
San Jose, California
JUNE 17–21, 201836
Spark 2.x – Effect of compression codec on query performance
San Jose, California
JUNE 17–21, 2018
37
Evaluation of Spark SQL with Spark 2.x versus Spark 1.6
• Hive testbench(similar to TPC/DS) with 1000 SF (1TB size) and 10000 SF
(10TB size) used for testing
• Hive testbench used to generate the data
• ORC format used for storing the data
• ANSI SQL compatibility
• Spark 2.x could run all Hive testbench queries whereas Spark 1.6 could run only
50 queries
• Spark SQL robustness
• With 10TB dataset size, Spark 2.x could finish all of the queries whereas Spark 1.6
could finish only about 40 queries
San Jose, California
JUNE 17–21, 2018
38
Spark 2.x performance improvements over Spark 1.6
- with 10000 SF (10TB)
San Jose, California
JUNE 17–21, 2018
39
Spark 2.x performance improvements over Spark 1.6
- with 10000 SF (10TB)
San Jose, California
JUNE 17–21, 2018
40
Spark 2.x - Scaling performance by adding Compute Nodes
without data rebalancing
San Jose, California
JUNE 17–21, 2018
41
https://spark.apache.org/
http://hpe.com/info/ra
http://hpe.com/info/sizers
http://hortonworks.com/partner/hpe
http://hortonworks.com/apache/spark
More Information
San Jose, California
JUNE 17–21, 201842
Questions ?
San Jose, California
JUNE 17–21, 2018
43
Mark Lochbihler, Hortonworks - Principal Architect
mlochbihler@hortonworks.com
Viplava Madasu, HPE - Big Data Systems Engineer
viplava.madasu@hpe.com

Más contenido relacionado

La actualidad más candente

Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
Building Community APIs using GraphQL, Neo4j, and Kotlin
Building Community APIs using GraphQL, Neo4j, and KotlinBuilding Community APIs using GraphQL, Neo4j, and Kotlin
Building Community APIs using GraphQL, Neo4j, and KotlinNeo4j
 
How to Burn Multi-GPUs using CUDA stress test memo
How to Burn Multi-GPUs using CUDA stress test memoHow to Burn Multi-GPUs using CUDA stress test memo
How to Burn Multi-GPUs using CUDA stress test memoNaoto MATSUMOTO
 
An Introduction to Apache Cassandra
An Introduction to Apache CassandraAn Introduction to Apache Cassandra
An Introduction to Apache CassandraSaeid Zebardast
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IODatabricks
 
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
 
NOSQL Databases types and Uses
NOSQL Databases types and UsesNOSQL Databases types and Uses
NOSQL Databases types and UsesSuvradeep Rudra
 
Deep dive into one drive known folder move
Deep dive into one drive known folder moveDeep dive into one drive known folder move
Deep dive into one drive known folder moveVlad Catrinescu
 
How to Build a Scylla Database Cluster that Fits Your Needs
How to Build a Scylla Database Cluster that Fits Your NeedsHow to Build a Scylla Database Cluster that Fits Your Needs
How to Build a Scylla Database Cluster that Fits Your NeedsScyllaDB
 
Apache doris (incubating) introduction
Apache doris (incubating) introductionApache doris (incubating) introduction
Apache doris (incubating) introductionleanderlee2
 
Productionzing ML Model Using MLflow Model Serving
Productionzing ML Model Using MLflow Model ServingProductionzing ML Model Using MLflow Model Serving
Productionzing ML Model Using MLflow Model ServingDatabricks
 
Intro to Elasticsearch
Intro to ElasticsearchIntro to Elasticsearch
Intro to ElasticsearchClifford James
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxData
 
Building a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache ArrowBuilding a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache ArrowDremio Corporation
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight OverviewJacques Nadeau
 
Top 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationshadooparchbook
 
Vitess VReplication: Standing on the Shoulders of a MySQL Giant
Vitess VReplication: Standing on the Shoulders of a MySQL GiantVitess VReplication: Standing on the Shoulders of a MySQL Giant
Vitess VReplication: Standing on the Shoulders of a MySQL GiantMatt Lord
 
Improving Apache Spark's Reliability with DataSourceV2
Improving Apache Spark's Reliability with DataSourceV2Improving Apache Spark's Reliability with DataSourceV2
Improving Apache Spark's Reliability with DataSourceV2Databricks
 

La actualidad más candente (20)

Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
Building Community APIs using GraphQL, Neo4j, and Kotlin
Building Community APIs using GraphQL, Neo4j, and KotlinBuilding Community APIs using GraphQL, Neo4j, and Kotlin
Building Community APIs using GraphQL, Neo4j, and Kotlin
 
How to Burn Multi-GPUs using CUDA stress test memo
How to Burn Multi-GPUs using CUDA stress test memoHow to Burn Multi-GPUs using CUDA stress test memo
How to Burn Multi-GPUs using CUDA stress test memo
 
An Introduction to Apache Cassandra
An Introduction to Apache CassandraAn Introduction to Apache Cassandra
An Introduction to Apache Cassandra
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IO
 
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...
 
Snowflake Overview
Snowflake OverviewSnowflake Overview
Snowflake Overview
 
NOSQL Databases types and Uses
NOSQL Databases types and UsesNOSQL Databases types and Uses
NOSQL Databases types and Uses
 
Deep dive into one drive known folder move
Deep dive into one drive known folder moveDeep dive into one drive known folder move
Deep dive into one drive known folder move
 
Introduction to Hadoop
Introduction to HadoopIntroduction to Hadoop
Introduction to Hadoop
 
How to Build a Scylla Database Cluster that Fits Your Needs
How to Build a Scylla Database Cluster that Fits Your NeedsHow to Build a Scylla Database Cluster that Fits Your Needs
How to Build a Scylla Database Cluster that Fits Your Needs
 
Apache doris (incubating) introduction
Apache doris (incubating) introductionApache doris (incubating) introduction
Apache doris (incubating) introduction
 
Productionzing ML Model Using MLflow Model Serving
Productionzing ML Model Using MLflow Model ServingProductionzing ML Model Using MLflow Model Serving
Productionzing ML Model Using MLflow Model Serving
 
Intro to Elasticsearch
Intro to ElasticsearchIntro to Elasticsearch
Intro to Elasticsearch
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
 
Building a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache ArrowBuilding a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache Arrow
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight Overview
 
Top 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applications
 
Vitess VReplication: Standing on the Shoulders of a MySQL Giant
Vitess VReplication: Standing on the Shoulders of a MySQL GiantVitess VReplication: Standing on the Shoulders of a MySQL Giant
Vitess VReplication: Standing on the Shoulders of a MySQL Giant
 
Improving Apache Spark's Reliability with DataSourceV2
Improving Apache Spark's Reliability with DataSourceV2Improving Apache Spark's Reliability with DataSourceV2
Improving Apache Spark's Reliability with DataSourceV2
 

Similar a Uncovering an Apache Spark 2 Benchmark - Configuration, Tuning and Test Results

Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov... Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...Databricks
 
Spark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca CanaliSpark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca CanaliSpark Summit
 
DUG'20: 02 - Accelerating apache spark with DAOS on Aurora
DUG'20: 02 - Accelerating apache spark with DAOS on AuroraDUG'20: 02 - Accelerating apache spark with DAOS on Aurora
DUG'20: 02 - Accelerating apache spark with DAOS on AuroraAndrey Kudryavtsev
 
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...inside-BigData.com
 
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...Agile Testing Alliance
 
Scaling Spark Workloads on YARN - Boulder/Denver July 2015
Scaling Spark Workloads on YARN - Boulder/Denver July 2015Scaling Spark Workloads on YARN - Boulder/Denver July 2015
Scaling Spark Workloads on YARN - Boulder/Denver July 2015Mac Moore
 
Big Telco Real-Time Network Analytics
Big Telco Real-Time Network AnalyticsBig Telco Real-Time Network Analytics
Big Telco Real-Time Network AnalyticsYousun Jeong
 
Big Telco - Yousun Jeong
Big Telco - Yousun JeongBig Telco - Yousun Jeong
Big Telco - Yousun JeongSpark Summit
 
Meetup Kubernetes Rhein-Necker
Meetup Kubernetes Rhein-NeckerMeetup Kubernetes Rhein-Necker
Meetup Kubernetes Rhein-Neckerinovex GmbH
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoMapR Technologies
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark FundamentalsZahra Eskandari
 
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmark
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmarkThe Apache Spark config behind the indsutry's first 100TB Spark SQL benchmark
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmarkLenovo Data Center
 
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)Claudiu Barbura
 
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay Rai
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay RaiConquering Hadoop and Apache Spark with Operational Intelligence with Akshay Rai
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay RaiDatabricks
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Databricks
 
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
 
xPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonxPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonClaudiu Barbura
 

Similar a Uncovering an Apache Spark 2 Benchmark - Configuration, Tuning and Test Results (20)

Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov... Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
Apache Spark for RDBMS Practitioners: How I Learned to Stop Worrying and Lov...
 
Spark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca CanaliSpark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca Canali
 
DUG'20: 02 - Accelerating apache spark with DAOS on Aurora
DUG'20: 02 - Accelerating apache spark with DAOS on AuroraDUG'20: 02 - Accelerating apache spark with DAOS on Aurora
DUG'20: 02 - Accelerating apache spark with DAOS on Aurora
 
Kafka & Hadoop in Rakuten
Kafka & Hadoop in RakutenKafka & Hadoop in Rakuten
Kafka & Hadoop in Rakuten
 
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...
Big Data Meets HPC - Exploiting HPC Technologies for Accelerating Big Data Pr...
 
APACHE SPARK.pptx
APACHE SPARK.pptxAPACHE SPARK.pptx
APACHE SPARK.pptx
 
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...
Introduction To Big Data with Hadoop and Spark - For Batch and Real Time Proc...
 
Scaling Spark Workloads on YARN - Boulder/Denver July 2015
Scaling Spark Workloads on YARN - Boulder/Denver July 2015Scaling Spark Workloads on YARN - Boulder/Denver July 2015
Scaling Spark Workloads on YARN - Boulder/Denver July 2015
 
Big Telco Real-Time Network Analytics
Big Telco Real-Time Network AnalyticsBig Telco Real-Time Network Analytics
Big Telco Real-Time Network Analytics
 
Big Telco - Yousun Jeong
Big Telco - Yousun JeongBig Telco - Yousun Jeong
Big Telco - Yousun Jeong
 
Meetup Kubernetes Rhein-Necker
Meetup Kubernetes Rhein-NeckerMeetup Kubernetes Rhein-Necker
Meetup Kubernetes Rhein-Necker
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of Twingo
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmark
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmarkThe Apache Spark config behind the indsutry's first 100TB Spark SQL benchmark
The Apache Spark config behind the indsutry's first 100TB Spark SQL benchmark
 
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)
xPatterns ... beyond Hadoop (Spark, Shark, Mesos, Tachyon)
 
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay Rai
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay RaiConquering Hadoop and Apache Spark with Operational Intelligence with Akshay Rai
Conquering Hadoop and Apache Spark with Operational Intelligence with Akshay Rai
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
 
Apache Spark PDF
Apache Spark PDFApache Spark PDF
Apache Spark PDF
 
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
 
xPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonxPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, Tachyon
 

Más de DataWorks Summit

Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisDataWorks Summit
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiDataWorks Summit
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...DataWorks Summit
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...DataWorks Summit
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal SystemDataWorks Summit
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExampleDataWorks Summit
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberDataWorks Summit
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixDataWorks Summit
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiDataWorks Summit
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsDataWorks Summit
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureDataWorks Summit
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EngineDataWorks Summit
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...DataWorks Summit
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudDataWorks Summit
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiDataWorks Summit
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerDataWorks Summit
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...DataWorks Summit
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouDataWorks Summit
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkDataWorks Summit
 

Más de DataWorks Summit (20)

Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache Ratis
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal System
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist Example
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at Uber
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability Improvements
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant Architecture
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything Engine
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google Cloud
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near You
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
 

Último

TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsNanddeep Nachan
 
Architecting Cloud Native Applications
Architecting Cloud Native ApplicationsArchitecting Cloud Native Applications
Architecting Cloud Native ApplicationsWSO2
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerThousandEyes
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FMESafe Software
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century educationjfdjdjcjdnsjd
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Angeliki Cooney
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...apidays
 
WSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdfSandro Moreira
 
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...apidays
 
DBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDropbox
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherRemote DBA Services
 
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelDeepika Singh
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...apidays
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...apidays
 
Six Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal OntologySix Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal Ontologyjohnbeverley2021
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfOrbitshub
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoffsammart93
 

Último (20)

TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectors
 
Architecting Cloud Native Applications
Architecting Cloud Native ApplicationsArchitecting Cloud Native Applications
Architecting Cloud Native Applications
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...
 
WSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering Developers
 
Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf
 
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
 
DBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor Presentation
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
 
Six Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal OntologySix Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal Ontology
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
 

Uncovering an Apache Spark 2 Benchmark - Configuration, Tuning and Test Results

  • 1. Uncovering an Apache Spark 2 Benchmark - Configuration, Tuning and Test Results • Mark Lochbihler, Hortonworks - Principal Architect • Viplava Madasu, HPE - Big Data Systems Engineer San Jose, California JUNE 17–21, 2018 1 Tuesday, June 19 4:00 PM - 4:40 PM Executive Ballroom 210C/D/G/H
  • 2. Today’s Agenda • What’s New with Spark 2.x – Mark • Spark Architecture • Spark on YARN • What’s New • Spark 2.x Benchmark - Viplava • What was Benchmarked • Configuration and Tuning • Infrastructure Used • Results • Questions / More Info – Mark and Viplava San Jose, California JUNE 17–21, 20182
  • 3. Apache Spark Apache Spark is a fast general-purpose engine for large-scale data processing. Spark was developed in response to limitations in Hadoop’s two-stage disk-based MapReduce processing framework. Orchestration: Spark’s standalone cluster manager, Apache Mesos, or Hadoop YARN San Jose, California JUNE 17–21, 2018 3
  • 4. Spark on Hadoop YARN YARN has the concept of labels for groupings of Hadoop Worker nodes. Spark on YARN is an optimal way to schedule and run Spark jobs on a Hadoop cluster alongside a variety of other data-processing frameworks, leveraging existing clusters using queue placement policies, and enabling security by running on Kerberos-enabled clusters. Client Mode Cluster Mode Client Executor App MasterSpark Driver Client Executor App Master Spark Driver San Jose, California JUNE 17–21, 20184
  • 5. Spark 2.x vs Spark 1.x Apache Spark 2.x is a major release update of Spark 1.x and includes significant updates in the following areas: • API usability • SQL 2003 support • Performance improvements • Structured streaming • R UDF support • Operational improvements San Jose, California JUNE 17–21, 2018 5
  • 6. Spark 2.x – New and Updated APIs Including: • Unifying DataFrame and Dataset APIs providing type safety for DataFrames • New SparkSession API with a new entry point that replaces the old SQLContext and HiveContext for DataFrame and Dataset APIs • New streamlined configuration API for SparkSession • New improved Aggregator API for typed aggregation in Datasets San Jose, California JUNE 17–21, 2018 6
  • 7. Spark 2.x – Improved SQL Functionality • ANSI SQL 2003 support • Enables running all 99 TPC-DS queries • A native SQL parser that supports both ANSI-SQL as well as Hive QL • Native DDL command implementations • Subquery support • Native CSV data source • Off-heap memory management for both caching and runtime execution • Hive-style bucketing support San Jose, California JUNE 17–21, 2018 7
  • 8. Spark 2.x – Performance Improvements • By implementing a new technique called “whole stage code generation”, Spark 2.x improves the performance 2-10 times for common operators in SQL and DataFrames. • Other performance improvements include: • Improved Parquet scan throughput through vectorization • Improved ORC performance • Many improvements in the Catalyst query optimizer for common workloads • Improved window function performance via native implementations for all window functions. San Jose, California JUNE 17–21, 2018 8
  • 9. Spark 2.x – Spark Machine Learning API • Spark 2.x replaces the RDD-based APIs in the spark.mllib package (put in maintenance mode) with the DataFrame-based API in the spark.ml package. • New features in the Spark 2.x Machine Learning API include: • ML persistence to support saving and loading ML models and Pipelines • New MLlib APIs in R for generalized linear models • Naive Bayes • K-Means Clustering • Survival regression • New MLlib APIs in Python for • LDA, Gaussian Mixture Model, Generalized Linear Regression, etc. San Jose, California JUNE 17–21, 2018 9
  • 10. Spark 2.x – Spark Streaming • Spark 2.x introduced a new high-level streaming API, called Structured Streaming, built on top of Spark SQL and the Catalyst optimizer. • Structured Streaming enables users to program against streaming sources and sinks using the same DataFrame/Dataset API as in static data sources, leveraging the Catalyst optimizer to automatically incrementalize the query plans. San Jose, California JUNE 17–21, 2018 10
  • 11. 11 Hortonworks Data Platform 2.6.5 – Just Released HDP 2.6.5 / 3.0 includes Apache Spark 2.3 ORC/Parquet Feature Parity – Spark extends its vectorized read capability to ORC data sources. – Structured streaming officially supports ORC data source with API and documentation Python Pandas UDF, with good performance and easy to use for Pandas users. This feature supports financial analysis use cases. Structured streaming now supports stream-stream joins. Structured streaming that goes to millisecond latency (Alpha). New continuous processing mode provides the best performance by minimizing the latency without waiting in idle status. San Jose, California JUNE 17–21, 2018
  • 12. Evaluation of Spark SQL with Spark 2.x versus Spark 1.6 • Benchmark Performed • Hive testbench, which is similar to TPC-DS benchmark • Tuning for the benchmark San Jose, California JUNE 17–21, 2018 12
  • 13. Why Cluster tuning matters • Spark/Hadoop default configurations are not optimal for most enterprise applications • Large number of configuration parameters • Tuning cluster will benefit all the applications • Can further tune job level configuration • More important if using disaggregated compute/storage layers as in HPE Reference Architecture • Useful for cloud too San Jose, California JUNE 17–21, 2018 13
  • 14. Factors to consider for Spark performance tuning • Hardware • CPU, Memory, Storage systems, Local disks, Network • Hadoop configuration • HDFS • YARN • Spark configuration • Executor cores, Executor memory, Shuffle partitions, Compression etc. San Jose, California JUNE 17–21, 2018 14
  • 15. General Hardware Guidelines • Sizing hardware for Spark depends on the use case, but Spark benefits from • More CPU cores • More memory • Flash storage for temporary storage • Faster network fabric • CPU Cores • Spark scales well to tens of CPU cores per machine • Most Spark applications are CPU bound, so at least 8-16 cores per machine. • Memory • Spark can make use of hundreds of gigabytes of memory per machine • Allocate only at most 75% of the memory for Spark; leave the rest for the operating system and buffer cache. • Storage tab of Spark’s monitoring UI will help. • Max 200GB per executor. San Jose, California JUNE 17–21, 2018 15
  • 16. General Hardware Guidelines … • Network • For Group-By, Reduce-By, and SQL join operations, network performance becomes important due to the Shuffles involved • 10 Gigabit network is the recommended choice • Local Disks • Spark uses local disks to store data that doesn’t fit in RAM, as well as to preserve intermediate output between stages • SSDs are recommended • Mount disks with noatime option to reduce unnecessary writes San Jose, California JUNE 17–21, 2018 16
  • 17. Spark on Hadoop/YARN (Cluster Mode) San Jose, California JUNE 17–21, 2018 17
  • 18. Useful HDFS configuration settings • Increase the dfs.blocksize value to allow more data to be processed by each map task • Also reduces NameNode memory consumption • dfs.blocksize 256/512MB • Increase the dfs.namenode.handler.count value to better manage multiple HDFS operations from multiple clients • dfs.namenode.handler.count 100 • To eliminate timeout exceptions (java.io.IOException: Unable to close file close file because the last block does not have enough number of replicas), San Jose, California JUNE 17–21, 2018 18
  • 19. Useful YARN configuration settings • YARN is the popular cluster manager for Spark on Hadoop, so it is important that YARN and Spark configurations are tuned in tandem. • Settings of Spark executor memory and executor cores result in allocation requests to YARN with the same values and YARN should be configured to accommodate the desired Spark settings • Amount of physical memory that can be allocated for containers per node • yarn.nodemanager.resource.memory-mb 384 GiB • Amount of vcores available on a compute node that can be allocated for containers • yarn.nodemanager.resource.cpu-vcores 48 San Jose, California JUNE 17–21, 2018 19
  • 20. YARN tuning … • Number of YARN containers depends on the nature of the workload • Assuming total of 384 GiB on each node, a workload that needs 24 GiB containers will result in 16 total containers • Assuming 12 worker nodes, number of 24 GiB containers = 16 * 12 – 1 = 191 • One container per YARN application master • General guideline is to configure containers in a way that maximizes the utilization of the memory and vcores on each node in the cluster San Jose, California JUNE 17–21, 2018 20
  • 21. YARN tuning … • Location of YARN intermediate files on the compute nodes • yarn.nodemanager.local-dirs /data1/hadoop/yarn/local, /data2/hadoop/yarn/local, /data3/hadoop/yarn/local, /data4/hadoop/yarn/local • Setting of spark.local.dir is ignored for YARN cluster mode • The node-locality-delay specifies how many scheduling intervals to let pass attempting to find a node local slot to run on prior to searching for a rack local slot • Important for small jobs that do not have a large number of tasks as it will better utilize the compute nodes • yarn.scheduler.capacity.node-locality-delay 1 San Jose, California JUNE 17–21, 2018 21
  • 22. Tuning Spark – Executor cores • Unlike Hadoop MapReduce where each map or reduce task is always started in a new process, Spark can efficiently use process threads (cores) to distribute task processing • Results in a need to tune Spark executors with respect to the amount of memory and number of cores each executor can use • Has to work within the configuration boundaries of YARN • Number of cores per executor can be controlled by • the configuration setting spark.executor.cores • the --executor-cores option of the spark-submit command • The default is 1 for Spark on YARN San Jose, California JUNE 17–21, 2018 22
  • 23. Tuning Spark – Executor cores • Simplest but inefficient approach would be to configure one executor per core and divide the memory equally among the number of executors • Since each partition cannot be computed on more than one executor, the size of each partition is limited and causes memory problems, or spilling to disk for shuffles • If the executors have only one core, then at most one task can run in each executor, which throws away the benefits of broadcast variables, which have to be sent to each executor once. • Each executor has some memory overhead (minimum of 384MB) – so, if we have many small executors, results in lot of memory overhead • Giving many cores to each executor also has issues • GC issues - since a larger JVM heap will delay the time until a GC event is triggered resulting in larger GC pauses • Results in poor HDSF throughput issues because of handling many concurrent threads • spark.executor.cores – experiment and set this based on your workloads. We found 9 was was the right setting for this configuration and bench test in our lab. San Jose, California JUNE 17–21, 2018 23
  • 24. Tuning Spark – Memory • Memory for each Spark job is application specific • Configure Executor memory in proportion to the number of partitions and cores per executor • Divide the total amount of memory on each node by the number of executors on the node • Should be less than the maximum YARN container size - so YARN maximum container size may need to be adjusted accordingly • Configuration setting spark.executor.memory or the --executor-memory option of the spark- submit command • JVM runs into issues with very large heaps (above 80GB). • Spark Driver memory • If driver collects too much data, the job may run into OOM errors. • Increase the driver memory using spark.driver.maxResultSize San Jose, California JUNE 17–21, 2018 24
  • 25. Spark 2.x – Memory Model • Each executor has memory overhead for things like VM overheads, interned strings, other native overheads • spark.yarn.executor.memoryOverhead • Default value is spark.executor.memory * 0.10, with minimum of 384MB. • Prior to Spark 1.6, separate tuning was needed for storage (RDD) memory and execution/shuffle memory via spark.storage.memoryFraction and spark.shuffle.memoryFraction • Spark 1.6 introduced a new “UnifiedMemoryManager” • When no Storage memory is used, Execution can acquire all the available memory and vice versa • As a result, applications that do not use caching can use the entire space for execution, obviating unnecessary disk spills. • Applications that do use caching can reserve a minimum storage space where their data blocks are immune to being evicted • spark.memory.storageFraction tunable, but good out-of-the-box performance San Jose, California JUNE 17–21, 2018 25
  • 26. Tuning Spark – Shuffle partitions • Spark SQL, by default, sets the number of reduce side partitions to 200 when doing a shuffle for wide transformations, e.g., groupByKey, reduceByKey, sortByKey etc. • Not optimal for many cases as it will use only 200 cores for processing tasks after the shuffle • For large datasets, this might result in shuffle block overflow resulting in job failures • The number of shuffle partitions should be at least equal to the number of total executor cores or a multiple of it in case of large data sets. • spark.sql.shuffle.partitions setting • Also – helps to partition in prime numbers in terms of hash effectiveness. San Jose, California JUNE 17–21, 2018 26
  • 27. Tuning Spark – Compression • Using compression in Spark can improve performance in a meaningful way as compression results in less disk I/O and network I/O • Even though compressing the data results in some CPU cycles being used, the performance improvements with compression outweigh the CPU overhead when a large amount of data is involved • Also compression results in reduced storage requirements for storing data on disk, e.g., intermediate shuffle files San Jose, California JUNE 17–21, 2018 27
  • 28. Tuning Spark – Compression • spark.io.compression.codec setting to decide the codec • three codecs provided: lz4, lzf, and snappy • default codec is lz4 • Four main places where Spark makes use of compression • Compress map output files during a shuffle operation using spark.shuffle.compress setting (Default true) • Compress data spilled during shuffles using spark.shuffle.spill.compress setting (Default true) • Compress broadcast variables before sending them using spark.broadcast.compress setting (Default true) • Compress serialized RDD partitions using spark.rdd.compress setting (Default false) San Jose, California JUNE 17–21, 2018 28
  • 29. Tuning Spark – Serialization type • Serialization plays an important role in the performance of any distributed application • Spark memory usage is greatly affected by storage level and serialization format • By default, Spark serializes objects using Java Serializer which can work with any class that implements java.io.Serializable interface • For custom data types, Kryo Serialization is more compact and efficient than Java Serialization • but user classes need to be explicitly registered with the Kryo Serializer • spark.serializer org.apache.spark.serializer.KryoSerializer • Spark SQL automatically uses Kryo serialization for DataFrames internally in Spark 2.x • For customer applications that still use RDDs, Kryo Serialization should result in a significant performance boost San Jose, California JUNE 17–21, 2018 29
  • 30. Tuning Spark – Other configuration settings • When using ORC/parquet format for the data, Spark SQL can push the filter down to ORC/parquet, thus avoiding large data transfer. • spark.sql.orc.filterPushdown (Default false) • spark.sql.parquet.filterPushdown (Default true) • For large data sets, you may encounter various network timeouts. Can tune different timeout values • spark.core.connection.ack.wait.timeout • spark.storage.blockManagerSlaveTimeoutMs • spark.shuffle.io.connectionTimeout • spark.rpc.askTimeout • spark.rpc.lookupTimeout • “Umbrella” setting for all these timeouts, spark.network.timeout (Default is 120 seconds). For 10TB dataset, this value should be something like 600 seconds. San Jose, California JUNE 17–21, 2018 30
  • 31. HPE’s Elastic Platform for Big Data Analytics (EPA) Modular building blocks of compute and storage optimized for modern workloads Apollo 2000 Compute DL360 Apollo 6500 w/ NVIDIA GPU Synergy Storage Apollo 4200DL380 Apollo 4510 Hot Cold Object Purpose - built Network FlexFabric 5950/5940 San Jose, California JUNE 17–21, 201831
  • 32. HPE EPA - Single-Rack Reference Architecture for Spark 2.x San Jose, California JUNE 17–21, 201832
  • 33. HPE EPA - Single-Rack Reference Architecture for Spark 2.x San Jose, California JUNE 17–21, 201833
  • 34. San Jose, California JUNE 17–21, 2018 Base Rack • (1) DL360 Control Block – (1) Management Node, (2) Head Nodes • (8) Apollo 2000 Compute Blocks – (32) XL170r Worker Nodes • (10) Apollo 4200 Storage Blocks – (10) Apollo 4200 Data Nodes • (1) Network Block • (1) Rack Block Aggregation Rack • (8) Apollo 2000 Compute Blocks – (32) XL170r Worker Nodes • (10) Apollo 4200 Storage Blocks – (10) Apollo 4200 Data Nodes • (1) Network Block • (1) Aggregation Switch Block - (2) HPE 5950 32QSFP28 • (1) Rack Block Expansion Rack • (8) Apollo 2000 Compute Blocks – (32) XL170r Worker Nodes • (10) Apollo 4200 Storage Blocks – (10) Apollo 4200 Data Nodes • (1) Network Block • (1) Rack Block Expansion Rack • (8) Apollo 2000 Compute Blocks – (32) XL170r Worker Nodes • (10) Apollo 4200 Storage Blocks – (10) Apollo 4200 Data Nodes • (1) Network Block • (1) Rack Block HPE EPA - Multi-Rack configuration 34
  • 35. Spark 2.x - Effect of cores per executor on query performance San Jose, California JUNE 17–21, 2018 35
  • 36. Spark 2.x – Effect of shuffle partitions on query performance San Jose, California JUNE 17–21, 201836
  • 37. Spark 2.x – Effect of compression codec on query performance San Jose, California JUNE 17–21, 2018 37
  • 38. Evaluation of Spark SQL with Spark 2.x versus Spark 1.6 • Hive testbench(similar to TPC/DS) with 1000 SF (1TB size) and 10000 SF (10TB size) used for testing • Hive testbench used to generate the data • ORC format used for storing the data • ANSI SQL compatibility • Spark 2.x could run all Hive testbench queries whereas Spark 1.6 could run only 50 queries • Spark SQL robustness • With 10TB dataset size, Spark 2.x could finish all of the queries whereas Spark 1.6 could finish only about 40 queries San Jose, California JUNE 17–21, 2018 38
  • 39. Spark 2.x performance improvements over Spark 1.6 - with 10000 SF (10TB) San Jose, California JUNE 17–21, 2018 39
  • 40. Spark 2.x performance improvements over Spark 1.6 - with 10000 SF (10TB) San Jose, California JUNE 17–21, 2018 40
  • 41. Spark 2.x - Scaling performance by adding Compute Nodes without data rebalancing San Jose, California JUNE 17–21, 2018 41
  • 43. Questions ? San Jose, California JUNE 17–21, 2018 43 Mark Lochbihler, Hortonworks - Principal Architect mlochbihler@hortonworks.com Viplava Madasu, HPE - Big Data Systems Engineer viplava.madasu@hpe.com

Notas del editor

  1. Edgeline
  2. Figure 4