+ All Categories
Home > Software > Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct...

Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct...

Date post: 11-Apr-2017
Category:
Upload: chris-fregly
View: 875 times
Download: 1 times
Share this document with a friend
51
How Spark Beat Hadoop @ 100 TB Sort + Project Tungsten Madrid Spark, Big Data, Bluemix Meetup Chris Fregly, Principal Data Solutions Engineer IBM Spark Technology Center Oct 22, 2015 Power of data. Simplicity of design. Speed of innovation. IBM | spark.tc
Transcript
Page 1: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

How Spark Beat Hadoop @ 100 TB Sort +

Project Tungsten

Madrid Spark, Big Data, Bluemix Meetup Chris Fregly, Principal Data Solutions Engineer

IBM Spark Technology Center Oct 22, 2015

Power of data. Simplicity of design. Speed of innovation. IBM | spark.tc

Page 2: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Who am I?!!

Streaming Data Engineer!Netflix Open Source Committer!

!Data Solutions Engineer!

Apache Contributor!!

Principal Data Solutions Engineer!IBM Technology Center!

Meetup Organizer!Advanced Apache Meetup!

Book Author!Advanced Spark (2016)!

Page 3: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Advanced Apache Spark Meetup Total Spark Experts: ~1400 in only 3 mos!!4th most active Spark Meetup in the world!!!Goals!

Dig deep into the Spark & extended-Spark codebase!!

Study integrations such as Cassandra, ElasticSearch,!Tachyon, S3, BlinkDB, Mesos, YARN, Kafka, R, etc!

!Surface and share the patterns and idioms of these !

well-designed, distributed, big data components!!

Page 4: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Freg-a-palooza Upcoming World Tour   London Spark Meetup (Oct 12th)!  Scotland Data Science Meetup (Oct 13th)!  Dublin Spark Meetup (Oct 15th)!  Barcelona Spark Meetup (Oct 20th)!  Madrid Spark/Big Data Meetup (Oct 22nd)!  Paris Spark Meetup (Oct 26th)!  Amsterdam Spark Summit (Oct 27th – Oct 29th)!  Delft Dutch Data Science Meetup (Oct 29th) !  Brussels Spark Meetup (Oct 30th)!  Zurich Big Data Developers Meetup (Nov 2nd)!

Page 5: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Daytona GraySort tChallenge sortbenchmark.org!

Page 6: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Topics of this Talk: Mechanical Sympathy!

Tungsten => Bare Metal!Seek Once, Scan Sequentially!!

CPU Cache Locality and Efficiency!

Use Data Structs Customized to Your Workload!

Go Off-Heap Whenever Possible !spark.unsafe.offHeap=true!

Page 7: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

What is the Daytona GraySort Challenge?!Key Metric!

Throughput of sorting 100TB of 100 byte data,10 byte key!Total time includes launching app and writing output file!

!Daytona!

App must be general purpose!!Gray!

Named after Jim Gray!

Page 8: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Input and Resources!Input!

Records are 100 bytes in length!First 10 bytes are random key!Input generator: ordinal.com/gensort.html!28,000 fixed-size partitions for 100 TB sort!250,000 fixed-size partitions for 1 PB sort!1 partition = 1 HDFS block = 1 executor !Aligned to avoid partial read I/O ie. imaginary data!

Hardware and Runtime Resources!Commercially available and off-the-shelf!Unmodified, no over/under-clocking!Generates 500TB of disk I/O, 200TB network I/O!

Page 9: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Rules!

Must sort to/from OS files in secondary storage!!No raw disk since I/O subsystem is being tested!!File and device striping (RAID 0) are encouraged!!Output file(s) must have correct key order!

Page 10: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Task Scheduling!Types of Data Locality!

PROCESS_LOCAL!NODE_LOCAL!RACK_LOCAL!ANY!

!Delay Scheduling!

spark.locality.wait.node: time to wait for next shitty level!Set to infinite to reduce shittiness, force NODE_LOCAL!Straggling Executor JVMs naturally fade away on each run!

Decreasing!Level of!Read !Performance!

Page 11: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Winning Results!On-disk only, in-memory caching disabled!!

EC2 (i2.8xlarge)! EC2 (i2.8xlarge)!

28,000!partitions!

250,000 !partitions (!!)!

(3 GBps/node!* 206 nodes)!

Page 12: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: EC2 Configuration!206 EC2 Worker nodes, 1 Master node!

AWS i2.8xlarge!32 Intel Xeon CPU E5-2670 @ 2.5 Ghz!244 GB RAM, 8 x 800GB SSD, RAID 0 striping, ext4!NOOP I/O scheduler: FIFO, request merging, no reordering!3 GBps mixed read/write disk I/O per node!

Deployed within Placement Group/VPC!Enhanced Networking!Single Root I/O Virtualization (SR-IOV): extension of PCIe!10 Gbps, low latency, low jitter (iperf showed ~9.5 Gbps)!

Page 13: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Winning Configuration!Spark 1.2, OpenJDK 1.7_<amazon-something>_u65-b17!Disabled in-memory caching -- all on-disk!!HDFS 2.4.1 short-circuit local reads, 2x replication!Writes flushed after each of the 5 runs!

28,000 partitions / (206 nodes * 32 cores) = 4.25 runs, round up 5 runs!Netty 4.0.23.Final with native epoll!Speculative Execution disabled: spark.speculation=false!Force NODE_LOCAL: spark.locality.wait.node=Infinite !Force Netty Off-Heap: spark.shuffle.io.preferDirectBuffers!Spilling disabled: spark.shuffle.spill=false!All compression disabled (network, on-disk, etc)!

Page 14: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Partitioning!Range Partitioning (vs. Hash Partitioning)!

Take advantage of sequential key space!Similar keys grouped together within a partition!Ranges defined by sampling 79 values per partition!Driver sorts samples and defines range boundaries!Sampling took ~10 seconds for 28,000 partitions!

!

Page 15: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Why Bother?!

Sorting relies heavily on shuffle, I/O subsystem!!Shuffle is major bottleneck in big data processing!

Large number of partitions can exhaust OS resources!!Shuffle optimization benefits all high-level libraries!!Goal is to saturate network controller on all nodes!

~125 MB/s (1 GB ethernet), 1.25 GB/s (10 GB ethernet)!

Page 16: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Per Node Results!

!!!!!Reducers: ~1.1 GB/s/node network I/O!

(max 1.25 Gbps for 10 GB ethernet)!Mappers: 3 GB/s/node disk I/O (8x800 SSD)!

206 nodes * 1.1 Gbps/node ~= 220 Gbps !

Page 17: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Quick Shuffle Refresher

Page 18: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

!!!!!!!!!!!

!

IBM | spark.tc

Shuffle Overview!

All to All, Cartesian Product Operation!

Least ->!Useful!Example!I Could!Find ->!

Page 19: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

!!!!!!!!!!!

!

IBM | spark.tc

Spark Shuffle Overview!

Most ->!Confusing!Example!I Could!Find ->!

Stages are Defined by Shuffle Boundaries!

Page 20: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Shuffle Intermediate Data: Spill to Disk!Intermediate shuffle data stored in memory!Spill to Disk!

spark.shuffle.spill=true!spark.shuffle.memoryFraction=% of all shuffle buffers!

Competes with spark.storage.memoryFraction!Bump this up from default!! Will help Spark SQL, too.!

Skipped Stages!Reuse intermediate shuffle data found on reducer!DAG for that partition can be truncated!

Page 21: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Shuffle Intermediate Data: Compression!

spark.shuffle.compress!Compress outputs (mapper)!

!spark.shuffle.spill.compress!

Compress spills (reducer)!!spark.io.compression.codec!

LZF: Most workloads (new default for Spark)!Snappy: LARGE workloads (less memory required to compress)!

Page 22: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Spark Shuffle Operations!

join!distinct!

cogroup!coalesce!repartition!sortByKey!

groupByKey!reduceByKey!

aggregateByKey!

Page 23: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Spark Shuffle Managers!spark.shuffle.manager = {!

hash < 10,000 Reducers!Output file determined by hashing the key of (K,V) pair!Each mapper creates an output buffer/file per reducer!Leads to M*R number of output buffers/files per shuffle!

sort >= 10,000 Reducers!Default since Spark 1.2!Wins Daytona GraySort Challenge w/ 250,000 reducers!!!

tungsten-sort -> Default in Spark 1.5!Uses com.misc.Unsafe for direct access to off heap!

}!

Page 24: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Shuffle Managers!

Page 25: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Shuffle Performance Tuning!Hash Shuffle Manager (no longer default)!

spark.shuffle.consolidateFiles: mapper output files!o.a.s.shuffle.FileShuffleBlockResolver!

Intermediate Files!Increase spark.shuffle.file.buffer: reduce seeks & sys calls!Increase spark.reducer.maxSizeInFlight if memory allows!Use smaller number of larger workers to reduce total files!

SQL: BroadcastHashJoin vs. ShuffledHashJoin!spark.sql.autoBroadcastJoinThreshold !Use DataFrame.explain(true) or EXPLAIN to verify!

Page 26: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Mechanical Sympathy

Page 27: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Mechanical Sympathy!Use as much of the CPU cache line as possible!!!!!!!!!!!!

Page 28: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Naïve Matrix Multiplication: Not Cache Friendly!Naive:!for (i = 0; i < N; ++i)! for (j = 0; j < N; ++j)! for (k = 0; k < N; ++k)! res[i][j] += mat1[i][k] * mat2[k][j];!

Clever: !double mat2transpose [N][N];!for (i = 0; i < N; ++i)! for (j = 0; j < N; ++j)! mat2transpose[i][j] = mat2[j][i];! for (i = 0; i < N; ++i)! for (j = 0; j < N; ++j)! for (k = 0; k < N; ++k)! res[i][j] += mat1[i][k] * mat2transpose[j][k];!

Prefetch Not Effective! On !

Row Wise Traversal!

Force All !Column Traversal by!Transposing Matrix 2!

Page 29: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Winning Optimizations

Deployed across Spark 1.1 and 1.2

Page 30: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Daytona GraySort Challenge: Winning Optimizations!CPU-Cache Locality: Mechanical Sympathy!

& Cache Locality/Alignment!!Optimized Sort Algorithm: Elements of (K, V) Pairs!!Reduce Network Overhead: Async Netty, epoll!!Reduce OS Resource Utilization: Sort Shuffle!

Page 31: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

CPU-Cache Locality: Mechanical Sympathy!AlphaSort paper ~1995!

Chris Nyberg and Jim Gray!!Naïve!

List (Pointer-to-Record)!Requires Key to be dereferenced for comparison!

!AlphaSort!

List (Key, Pointer-to-Record)!Key is directly available for comparison!

!

Key! Ptr!

Ptr!

Page 32: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

CPU-Cache Locality: Cache Locality/Alignment!Key(10 bytes) + Pointer(4 bytes*) = 14 bytes!

*4 bytes when using compressed OOPS (<32 GB heap)!Not binary in size!Not CPU-cache friendly!

Cache Alignment Options! Add Padding (2 bytes)!

Key(10 bytes) + Pad(2 bytes) + Pointer(4 bytes)=16 bytes!

(Key-Prefix, Pointer-to-Record)!Key distribution affects performance! Prefix (4 bytes) + Pointer (4 bytes) = 8 bytes!

Key!

Key!

Ptr!

Ptr!

Ptr!Key-Prefx!

Pad!

With Padding!

Cache-line!Friendly!

Page 33: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

CPU-Cache Locality: Performance Comparison!

Page 34: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Similar Technique: Direct Cache Access!

^ Packet header placed into CPU cache ^!

Page 35: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Optimized Sort Algorithm: Elements of (K, V) Pairs!o.a.s.util.collection.TimSort!

Based on JDK 1.7 TimSort!Performs best on partially-sorted datasets !Optimized for elements of (K,V) pairs!Sorts impl of SortDataFormat (ie. KVArraySortDataFormat)!

!o.a.s.util.collection.AppendOnlyMap!

Open addressing hash, quadratic probing!Array of [(K, V), (K, V)] !Good memory locality!Keys never removed, values only append!

(^2 Probing)!

Page 36: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Reduce Network Overhead: Async Netty, epoll!

New Network Module based on Async Netty!Replaces old java.nio, low-level, socket-based code!Zero-copy epoll uses kernel-space between disk & network!Custom memory management reduces GC pauses!spark.shuffle.blockTransferService=netty!

Spark-Netty Performance Tuning!spark.shuffle.io.numConnectionsPerPeer!

Increase to saturate hosts with multiple disks!spark.shuffle.io.preferDirectBuffers!

On or Off-heap (Off-heap is default)!

Page 37: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Hash Shuffle Manager!!!!!!!!!!!!

M*R num open files per shuffle; M=num mappers! R=num reducers!

Mapper Opens 1 File per Partition/Reducer!HDFS!

(2x repl)!

HDFS!(2x repl)!

Page 38: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

S!

IBM | spark.tc

Reduce OS Resource Utilization: Sort Shuffle!

!!!!!!!!

M open files per shuffle; M = num of mappers!spark.shuffle.sort.bypassMergeThreshold!

Merge Sort!(Disk)!

Reducers seek and scan from range offset!

of Master File on Mapper!

TimSort!(RAM)!

HDFS!(2x repl)!

HDFS!(2x repl)!

SPARK-2926:! Replace

TimSort w/Merge Sort!(Memory)!

Mapper Merge Sorts Partitions into 1 Master File Indexed by Partition Range Offsets!

<- Master->! File!

Page 39: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Project Tungsten

Deployed across Spark 1.4 and 1.5

Page 40: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Significant Spark Core Changes!Disk!

Network!CPU!

Memory!

Daytona GraySort Optimizations!(Spark 1.1-1.2, Late 2014)!

Tungsten Optimizations!(Spark 1.4-1.5, Late 2015)!

Page 41: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Why is CPU the Bottleneck?!Network and Disk I/O bandwidth are relatively high!!GraySort optimizations improved network & shuffle!!Predicate pushdowns and partition pruning!!Columnar file formats like Parquet and ORC!!CPU used for serialization, hashing, compression!

Page 42: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

tungsten-sort Shuffle Manager!

“I don’t know your data structure, but my array[] will beat it!” Custom Data Structures for Sort/Shuffle Workload!UnsafeRow: !!!!

Rows are !8-byte aligned!

Primitives are inlined!Row.equals(), Row.hashCode()!

operate on raw bytes!Offset (Int) and Length (Int)!

Stored in a single Long!

Page 43: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

sun.misc.Unsafe!Info!

addressSize()!pageSize()!

Objects!allocateInstance()!objectFieldOffset()!

Classes!staticFieldOffset()!defineClass()!defineAnonymousClass()!ensureClassInitialized()!

Synchronization!monitorEnter()!tryMonitorEnter()!monitorExit()!compareAndSwapInt()!putOrderedInt()!

Arrays!arrayBaseOffset()!arrayIndexScale()!

Memory!allocateMemory()!copyMemory()!freeMemory()!getAddress() – not guaranteed correct if GC occurs!getInt()/putInt()!getBoolean()/putBoolean()!getByte()/putByte()!getShort()/putShort()!getLong()/putLong()!getFloat()/putFloat()!getDouble()/putDouble()!getObjectVolatile()/putObjectVolatile()!

Used by Spark!

Page 44: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Spark + com.misc.Unsafe!org.apache.spark.sql.execution.!

aggregate.SortBasedAggregate!aggregate.TungstenAggregate!aggregate.AggregationIterator!aggregate.udaf!aggregate.utils!SparkPlanner!rowFormatConverters!UnsafeFixedWidthAggregationMap!UnsafeExternalSorter!UnsafeExternalRowSorter!UnsafeKeyValueSorter!UnsafeKVExternalSorter!local.ConvertToUnsafeNode!local.ConvertToSafeNode!local.HashJoinNode!local.ProjectNode!local.LocalNode!local.BinaryHashJoinNode!local.NestedLoopJoinNode!joins.HashJoin!joins.HashSemiJoin!joins.HashedRelation!joins.BroadcastHashJoin!joins.ShuffledHashOuterJoin (not yet converted)!joins.BroadcastHashOuterJoin!joins.BroadcastLeftSemiJoinHash!joins.BroadcastNestedLoopJoin!joins.SortMergeJoin!joins.LeftSemiJoinBNL!joins.SortMergerOuterJoin!Exchange!SparkPlan!UnsafeRowSerializer!SortPrefixUtils!sort!basicOperators!aggregate.SortBasedAggregationIterator!aggregate.TungstenAggregationIterator!datasources.WriterContainer!datasources.json.JacksonParser!datasources.jdbc.JDBCRDD!Window!

org.apache.spark.!unsafe.Platform!unsafe.KVIterator!unsafe.array.LongArray!unsafe.array.ByteArrayMethods!unsafe.array.BitSet!unsafe.bitset.BitSetMethods!unsafe.hash.Murmur3_x86_32!unsafe.map.BytesToBytesMap!unsafe.map.HashMapGrowthStrategy!unsafe.memory.TaskMemoryManager!unsafe.memory.ExecutorMemoryManager!unsafe.memory.MemoryLocation!unsafe.memory.UnsafeMemoryAllocator!unsafe.memory.MemoryAllocator (trait/interface)!unsafe.memory.MemoryBlock!unsafe.memory.HeapMemoryAllocator!unsafe.memory.ExecutorMemoryManager!unsafe.sort.RecordComparator!unsafe.sort.PrefixComparator!unsafe.sort.PrefixComparators!unsafe.sort.UnsafeSorterSpillWriter!serializer.DummySerializationInstance!shuffle.unsafe.UnsafeShuffleManager!shuffle.unsafe.UnsafeShuffleSortDataFormat!shuffle.unsafe.SpillInfo!shuffle.unsafe.UnsafeShuffleWriter!shuffle.unsafe.UnsafeShuffleExternalSorter!shuffle.unsafe.PackedRecordPointer!shuffle.ShuffleMemoryManager!util.collection.unsafe.sort.UnsafeSorterSpillMerger!util.collection.unsafe.sort.UnsafeSorterSpillReader!util.collection.unsafe.sort.UnsafeSorterSpillWriter!util.collection.unsafe.sort.UnsafeShuffleInMemorySorter!util.collection.unsafe.sort.UnsafeInMemorySorter!util.collection.unsafe.sort.RecordPointerAndKeyPrefix!util.collection.unsafe.sort.UnsafeSorterIterator!network.shuffle.ExternalShuffleBlockResolver!scheduler.Task!rdd.SqlNewHadoopRDD!executor.Executor!

org.apache.spark.sql.catalyst.expressions.!regexpExpressions!BoundAttribute!SortOrder!SpecializedGetters!ExpressionEvalHelper!UnsafeArrayData!UnsafeReaders!UnsafeMapData!Projection!LiteralGeneartor!UnsafeRow!JoinedRow!SpecializedGetters!InputFileName!SpecificMutableRow!codegen.CodeGenerator!codegen.GenerateProjection!codegen.GenerateUnsafeRowJoiner!codegen.GenerateSafeProjection!codegen.GenerateUnsafeProjection!codegen.BufferHolder!codegen.UnsafeRowWriter!codegen.UnsafeArrayWriter!complexTypeCreator!rows!literals!misc!stringExpressions!

Over 200 source!files affected!!!

Page 45: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

CPU and Memory Optimizations!Custom Managed Memory

Reduces GC overhead Both on and off heap Exact size calculations

Direct Binary Processing Operate on serialized/compressed arrays Kryo can reorder serialized records LZF can reorder compressed records

More CPU Cache-aware Data Structs & Algorithms o.a.s.unsafe.map.BytesToBytesMap vs. j.u.HashMap

Code Generation (default in 1.5) Generate source code from overall query plan Janino generates bytecode from source code 100+ UDFs converted to use code generation

Details in !SPARK-7075!

UnsafeFixedWithAggregationMap,& !TungstenAggregationIterator!

CodeGenerator &! GeneratorUnsafeRowJoiner!UnsafeSortDataFormat &!

UnsafeShuffleSortDataFormat &!PackedRecordPointer &!

UnsafeRow!

UnsafeInMemorySorter & !UnsafeExternalSorter & !

UnsafeShuffleWriter!

Mostly Same Join Code,!added if (isUnsafeMode)!

UnsafeShuffleManager &!UnsafeShuffleInMemorySorter &!

UnsafeShuffleExternalSorter!

Page 46: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Code Generation!Turned on by default in Spark 1.5 Problem: Generic expression evaluation Expensive on JVM Virtual func calls Branches based on expression type Excessive object creation due to primitive boxing

Implementation Defer the source code generation to each operator, type, etc Scala quasiquotes provide Scala AST manipulation/rewriting Generated source code is compiled to bytecode w/ Janino 100+ UDFs now using code gen

Page 47: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Code Generation: Spark SQL UDFs!100+ UDFs now using code gen – More to come in Spark 1.6!

Details in !SPARK-8159!

Page 48: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Project Tungsten: Beyond Core and Spark SQL!

SortDataFormat<K, Buffer>: Base trait UncompressedInBlockSort: MLlib.ALS EdgeArraySortDataFormat: GraphX.Edge

Page 49: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

IBM | spark.tc

Relevant Links!  http://sortbenchmark.org/ApacheSpark2014.pdf!

!  https://databricks.com/blog/2014/11/05/spark-officially-sets-a-new-record-in-large-scale-sorting.html!

  http://0x0fff.com/spark-architecture-shuffle/!

  http://www.cs.berkeley.edu/~kubitron/courses/cs262a-F13/projects/reports/project16_report.pdf!

  http://stackoverflow.com/questions/763262/how-does-one-write-code-that-best-utilizes-the-cpu-cache-to-improve-performance!

  http://www.aristeia.com/TalkNotes/ACCU2011_CPUCaches.pdf!

  http://mishadoff.com/blog/java-magic-part-4-sun-dot-misc-dot-unsafe/!

  http://docs.scala-lang.org/overviews/quasiquotes/intro.html!!  http://lwn.net/Articles/252125/ <-- Memory Part 2: CPU Caches!

  http://lwn.net/Articles/255364/ <-- Memory Part 5: What Programmers Can Do!

!

Page 50: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Signup for the book and meetup! advancedspark.com

Clone all code used today!

github.com/fluxcapacitor/pipeline

Run all demos presented today! hub.docker.com/r/fluxcapacitor/pipeline

IBM | spark.tc

Sign up for our newsletter at

Thank You, Madrid!!

Page 51: Madrid Spark Big Data Bluemix Meetup - Spark Versus Hadoop @ 100 TB Daytona GraySort Challenge - Oct 22, 2015

Power of data. Simplicity of design. Speed of innovation.

IBM Spark


Recommended