hadoop summit europe 2014: apache storm architecture

113
© Hortonworks Inc. 2011 P. Taylor Goetz Apache Storm Committer [email protected] @ptgoetz Apache Storm Architecture and Integration Real-Time Big Data

Upload: p-taylor-goetz

Post on 27-Aug-2014

1.800 views

Category:

Software


3 download

DESCRIPTION

The slides from my session on Apache Storm architecture at Hadoop Summit Europe 2014.

TRANSCRIPT

Page 1: Hadoop Summit Europe 2014: Apache Storm Architecture

© Hortonworks Inc. 2011

P. Taylor Goetz Apache Storm Committer [email protected] @ptgoetz

Apache Storm Architecture and Integration

Real-Time Big Data

Page 2: Hadoop Summit Europe 2014: Apache Storm Architecture

Shedding Light on Data

Page 3: Hadoop Summit Europe 2014: Apache Storm Architecture

Shedding Light on Big Data

Page 4: Hadoop Summit Europe 2014: Apache Storm Architecture

Shedding Light on Big Data In Real Time

Page 5: Hadoop Summit Europe 2014: Apache Storm Architecture

What is Storm?

Page 6: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Streaming

Page 7: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is StreamingKey enabler of the Lamda Architecture

Page 8: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Fast

Page 9: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is FastClocked at 1M+ messages per second per node

Page 10: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Scalable

Page 11: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is ScalableThousands of workers per cluster

Page 12: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Fault Tolerant

Page 13: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Fault TolerantFailure is expected, and embraced

Page 14: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is Reliable

Page 15: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is ReliableGuaranteed message delivery

Page 16: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm is ReliableExactly-once semantics

Page 17: Hadoop Summit Europe 2014: Apache Storm Architecture

Conceptual Model

Page 18: Hadoop Summit Europe 2014: Apache Storm Architecture

Tuple

{…}

Page 19: Hadoop Summit Europe 2014: Apache Storm Architecture

Tuple

{…} • Core Unit of Data • Immutable Set of Key/Value

Pairs

Page 20: Hadoop Summit Europe 2014: Apache Storm Architecture

Streams

{…} {…} {…} {…} {…} {…} {…}

Unbounded Sequence of Tuples

Page 21: Hadoop Summit Europe 2014: Apache Storm Architecture

Spouts

Page 22: Hadoop Summit Europe 2014: Apache Storm Architecture

Spouts

• Source of Streams • Wraps a streaming data source

and emits Tuples

{…}{…}

{…}{…}

{…}{…}

{…}

{…} {…} {…} {…} {…} {…} {…}

Page 23: Hadoop Summit Europe 2014: Apache Storm Architecture

Spout APIpublic interface ISpout extends Serializable {!! void open(Map conf, !! TopologyContext context, !! ! ! SpoutOutputCollector collector);!! void close();! ! void activate();! ! void deactivate();!! void nextTuple();!! void ack(Object msgId);!! void fail(Object msgId);!}

Lifecycle API

Page 24: Hadoop Summit Europe 2014: Apache Storm Architecture

Spout APIpublic interface ISpout extends Serializable {!! void open(Map conf, !! TopologyContext context, !! ! ! SpoutOutputCollector collector);!! void close();! ! void activate();! ! void deactivate();!! void nextTuple();!! void ack(Object msgId);!! void fail(Object msgId);!}

Core API

Page 25: Hadoop Summit Europe 2014: Apache Storm Architecture

Spout APIpublic interface ISpout extends Serializable {!! void open(Map conf, !! TopologyContext context, !! ! ! SpoutOutputCollector collector);!! void close();! ! void activate();! ! void deactivate();!! void nextTuple();!! void ack(Object msgId);!! void fail(Object msgId);!}

Reliability API

Page 26: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolts

Page 27: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolts

• Core functions of a streaming computation

• Receive tuples and do stuff • Optionally emit additional

tuples

Page 28: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolts

• Write to a data store

Page 29: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolts

• Read from a data store

Page 30: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolts

• Perform arbitrary computation

Compute

Page 31: Hadoop Summit Europe 2014: Apache Storm Architecture

{…}{…}

{…}{…}

{…}{…}

{…}

Bolts

• (Optionally) Emit additional streams

{…} {…} {…} {…} {…} {…} {…}

Page 32: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolt API

public interface IBolt extends Serializable {!! void prepare(Map stormConf, ! TopologyContext context, ! OutputCollector collector);!! void cleanup();!! ! void execute(Tuple input);!! !}

Lifecycle API

Page 33: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolt API

public interface IBolt extends Serializable {!! void prepare(Map stormConf, ! TopologyContext context, ! OutputCollector collector);!! void cleanup();!! ! void execute(Tuple input);!! !}

Core API

Page 34: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolt Output API

public interface IOutputCollector extends IErrorReporter {!! List<Integer> emit(String streamId, ! Collection<Tuple> anchors, ! List<Object> tuple);!! ! void emitDirect(int taskId, ! String streamId, ! Collection<Tuple> anchors, ! List<Object> tuple);!! ! void ack(Tuple input);!! ! void fail(Tuple input);!}

Core API

Page 35: Hadoop Summit Europe 2014: Apache Storm Architecture

Bolt Output API

public interface IOutputCollector extends IErrorReporter {!! List<Integer> emit(String streamId, ! Collection<Tuple> anchors, ! List<Object> tuple);!! ! void emitDirect(int taskId, ! String streamId, ! Collection<Tuple> anchors, ! List<Object> tuple);!! ! void ack(Tuple input);!! ! void fail(Tuple input);!}

Reliability API

Page 36: Hadoop Summit Europe 2014: Apache Storm Architecture

Topologies

Page 37: Hadoop Summit Europe 2014: Apache Storm Architecture

Topologies

Page 38: Hadoop Summit Europe 2014: Apache Storm Architecture

Topologies

• DAG of Spouts and Bolts • Data Flow Representation • Streaming Computation

Page 39: Hadoop Summit Europe 2014: Apache Storm Architecture

Topologies

• Storm executes spouts and bolts as individual Tasks that run in parallel on multiple machines.

Page 40: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

Page 41: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

Stream Groupings determine how Storm routes Tuples between tasks in a topology

Page 42: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

Shuffle!!

Randomized round-robin.

Page 43: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

LocalOrShuffle!!

Randomized round-robin. (With a preference for intra-worker Tasks)

Page 44: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

Fields Grouping!!

Ensures all Tuples with with the same field value(s) are always routed to the same task.

Page 45: Hadoop Summit Europe 2014: Apache Storm Architecture

Stream Groupings

Fields Grouping!!

Ensures all Tuples with with the same field value(s) are always routed to the same task.

!(this is a simple hash of the field values,

modulo the number of tasks)

Page 46: Hadoop Summit Europe 2014: Apache Storm Architecture

Physical View

Page 47: Hadoop Summit Europe 2014: Apache Storm Architecture

Physical ViewZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

Worker* Worker* Worker* Worker*

Page 48: Hadoop Summit Europe 2014: Apache Storm Architecture

Topology Deployment

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Topology Submitter uploads topology:!• topology.jar!• topology.ser!• conf.ser

$ bin/storm jar

Page 49: Hadoop Summit Europe 2014: Apache Storm Architecture

Topology Deployment

Nimbus calculates assignments and sends to Zookeeper

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Page 50: Hadoop Summit Europe 2014: Apache Storm Architecture

Topology Deployment

Supervisor nodes receive assignment information !via Zookeeper watches.

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Page 51: Hadoop Summit Europe 2014: Apache Storm Architecture

Topology Deployment

Supervisor nodes download topology from Nimbus:!• topology.jar!• topology.ser!• conf.ser

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Page 52: Hadoop Summit Europe 2014: Apache Storm Architecture

Topology Deployment

Supervisors spawn workers (JVM processes) to start the topology

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker Worker

Page 53: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

Page 54: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

Workers heartbeat back to Supervisors and Nimbus via ZooKeeper, !as well as locally.

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker Worker

Page 55: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

If a worker dies (fails to heartbeat), the Supervisor will restart it

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker WorkerX

Page 56: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

If a worker dies repeatedly, Nimbus will reassign the work to other!nodes in the cluster.

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker WorkerX

Page 57: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

If a supervisor node dies, Nimbus will reassign the work to other nodes.

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker WorkerXX

Page 58: Hadoop Summit Europe 2014: Apache Storm Architecture

Fault Tolerance

If Nimbus dies, topologies will continue to function normally,!but won’t be able to perform reassignments.

ZooKeeperNimbus

Supervisor Supervisor Supervisor Supervisor

TopologySubmitter

Worker Worker Worker Worker

X

Page 59: Hadoop Summit Europe 2014: Apache Storm Architecture

ParallelismScaling a Distributed Computation

Page 60: Hadoop Summit Europe 2014: Apache Storm Architecture

Parallelism

Worker (JVM)

Executor (Thread) Executor (Thread) Executor (Thread)

Task Task Task

1 Worker, Parallelism = 1

Page 61: Hadoop Summit Europe 2014: Apache Storm Architecture

ParallelismWorker (JVM)

Executor (Thread) Executor (Thread) Executor (Thread)

Task Task Task

Executor (Thread)

Task

1 Worker, Parallelism = 2

Page 62: Hadoop Summit Europe 2014: Apache Storm Architecture

ParallelismWorker (JVM)

Executor (Thread) Executor (Thread)

Task Task

Executor (Thread)

Task

Task

1 Worker, Parallelism = 2, NumTasks = 2

Page 63: Hadoop Summit Europe 2014: Apache Storm Architecture

Parallelism

3 Workers, Parallelism = 1, NumTasks = 1

Worker (JVM)Worker (JVM)Worker (JVM)

Executor (Thread) Executor (Thread) Executor (Thread)

Task Task Task

Page 64: Hadoop Summit Europe 2014: Apache Storm Architecture

Internal Messaging

Page 65: Hadoop Summit Europe 2014: Apache Storm Architecture

Internal MessagingWorker Mechanics

Page 66: Hadoop Summit Europe 2014: Apache Storm Architecture

Worker Internal Messaging

Worker Receive Thread

Worker Port

List<List<Tuple>>Receive Buffer

Executor Thread *

Inbound Queue Outbound Queue

Router Send Thread

Worker Transfer Thread

List<List<Tuple>>Transfer Buffer

To Other Workers

Task(Spout/Bolt)

Task(Spout/Bolt)

Task(s)(Spout/Bolt)

Page 67: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable ProcessingAt Least Once

Page 68: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Bolts may emit Tuples Anchored to one received. Tuple “B” is a descendant of Tuple “A”

{A} {B}

Page 69: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Multiple Anchorings form a Tuple tree (bolts not shown)

{A} {B}

{C}

{D}

{E}

{F}

{G}

{H}

Page 70: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Bolts can Acknowledge that a tuple has been processed successfully.

{A} {B}

ACK

Page 71: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Acks are delivered via a system-level bolt

ACK

{A} {B}

Acker Bolt

ackack

Page 72: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Bolts can also Fail a tuple to trigger a spout to replay the original.

FAIL

{A} {B}

Acker Bolt

failfail

Page 73: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

Any failure in the Tuple tree will trigger a replay of the original tuple

{A} {B}

{C}

{D}

{E}

{F}

{G}

{H}

X

X

Page 74: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

How to track a large-scale tuple tree efficiently?

Page 75: Hadoop Summit Europe 2014: Apache Storm Architecture

Reliable Processing

A single 64-bit integer.

Page 76: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR Magic

Long a, b, c = Random.nextLong();

Page 77: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR Magic

Long a, b, c = Random.nextLong();!!a ^ a == 0

Page 78: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR Magic

Long a, b, c = Random.nextLong();!!a ^ a == 0!!a ^ a ^ b != 0

Page 79: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR MagicLong a, b, c = Random.nextLong();!!a ^ a == 0!!a ^ a ^ b != 0!!a ^ a ^ b ^ b == 0

Page 80: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR Magic

Long a, b, c = Random.nextLong();!!a ^ (a ^ b) ^ c ^ (b ^ c) == 0

Page 81: Hadoop Summit Europe 2014: Apache Storm Architecture

XOR Magic

Long a, b, c = Random.nextLong();!!a ^ (a ^ b) ^ c ^ (b ^ c) == 0

Acks can arrive asynchronously, in any order

Page 82: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Page 83: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

High-level abstraction built on Storm’s core primitives.

Page 84: Hadoop Summit Europe 2014: Apache Storm Architecture

TridentBuilt-in support for:

• Merges and Joins

• Aggregations

• Groupings

• Functions

• Filters

Page 85: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Stateful, incremental processing on top of any persistence store.

Page 86: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Trident is Storm

Page 87: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Fluent, Stream-oriented API

Page 88: Hadoop Summit Europe 2014: Apache Storm Architecture

TridentFluent, Stream-Oriented API

TridentTopology topology = new TridentTopology();!FixedBatchSpout spout = new FixedBatchSpout(…);!Stream stream = topology.newStream("words", spout);!!stream.each(…, new MyFunction())! .groupBy()! .each(…, new MyFilter())! .persistentAggregate(…);!

User-defined functions

Page 89: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Micro-Batch Oriented

Tuple Micro-Batch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

Page 90: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Trident Batches are Ordered

Tuple Micro-Batch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

Tuple Micro-Batch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

Batch #1 Batch #2

Page 91: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Trident Batches can be Partitioned

Tuple Micro-Batch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

Page 92: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident

Trident Batches can be Partitioned

Tuple Micro-Batch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

Partition Operation

Partition A

{…} {…}

{…}{…}

Partition B

{…} {…}

{…}{…}

Partition C

{…} {…}

{…}{…}

Partition D

{…} {…}

{…}{…}

Page 93: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Operation Types

1. Local Operations (Functions/Filters)

2. Repartitioning Operations (Stream Groupings, etc.)

3. Aggregations

4. Merges/Joins

Page 94: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Topologies

each

each

shuffle

Function

Filter

partition persist

Page 95: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Toplogies

Partitioning operations define the boundaries between bolts, and thus network transfer

and parallelism

Page 96: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Topologies

each

each

shuffle

Function

Filter

partition persist

Bolt 1

Bolt 2

shuffleGrouping()

Partitioning!Operation

Page 97: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Batch Coordination

Page 98: Hadoop Summit Europe 2014: Apache Storm Architecture

Trident Batch Coordination

Trident SpoutMaster Batch Coordinator User Logic

nextbatch

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

{…} {…} {…} {…}

commit

Page 99: Hadoop Summit Europe 2014: Apache Storm Architecture

Controlling Deployment

Page 100: Hadoop Summit Europe 2014: Apache Storm Architecture

Controlling Deployment

How do you control where spouts and bolts get deployed in a cluster?

Page 101: Hadoop Summit Europe 2014: Apache Storm Architecture

Controlling Deployment

How do you control where spouts and bolts get deployed in a cluster?

Plug-able Schedulers

Page 102: Hadoop Summit Europe 2014: Apache Storm Architecture

Controlling Deployment

How do you control where spouts and bolts get deployed in a cluster?

Isolation Scheduler

Page 103: Hadoop Summit Europe 2014: Apache Storm Architecture

Wait… Nimbus, Supervisor, Schedulers… !

Doesn’t that sound kind of like resource negotiation?

Page 104: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

HDFS2  (redundant,  reliable  storage)

YARN  (cluster  resource  management)

MapReduce (batch)

Apache  STORM  (streaming)

HADOOP 2.0

Tez  (interactive)

Multi Use Data Platform Batch, Interactive, Online, Streaming, …

Page 105: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

HDFS2  (redundant,  reliable  storage)

YARN  (cluster  resource  management)

MapReduce (batch)

Apache  STORM  (streaming)

HADOOP 2.0

Tez  (interactive)

Multi Use Data Platform Batch, Interactive, Online, Streaming, …

Batch and real-time on the same cluster

Page 106: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

HDFS2  (redundant,  reliable  storage)

YARN  (cluster  resource  management)

MapReduce (batch)

Apache  STORM  (streaming)

HADOOP 2.0

Tez  (interactive)

Multi Use Data Platform Batch, Interactive, Online, Streaming, …

Security and Multi-tenancy

Page 107: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

HDFS2  (redundant,  reliable  storage)

YARN  (cluster  resource  management)

MapReduce (batch)

Apache  STORM  (streaming)

HADOOP 2.0

Tez  (interactive)

Multi Use Data Platform Batch, Interactive, Online, Streaming, …

Elasticity

Page 108: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

Nimbus Resource Management, Scheduling

Supervisor Node and Process management

Workers Runs topology tasks

YARN RM Resource Management

Storm AM Manage Topology

Containers Runs topology tasks

YARN NM Process Management

Storm’s resource management system maps very naturally to the YARN model.

Page 109: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

Nimbus Resource Management, Scheduling

Supervisor Node and Process management

Workers Runs topology tasks

YARN RM Resource Management

Storm AM Manage Topology

Containers Runs topology tasks

YARN NM Process Management

High Availability

Page 110: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

Nimbus Resource Management, Scheduling

Supervisor Node and Process management

Workers Runs topology tasks

YARN RM Resource Management

Storm AM Manage Topology

Containers Runs topology tasks

YARN NM Process Management

Detect and scale around bottlenecks

Page 111: Hadoop Summit Europe 2014: Apache Storm Architecture

Storm on YARN

Nimbus Resource Management, Scheduling

Supervisor Node and Process management

Workers Runs topology tasks

YARN RM Resource Management

Storm AM Manage Topology

Containers Runs topology tasks

YARN NM Process Management

Optimize for available resources

Page 112: Hadoop Summit Europe 2014: Apache Storm Architecture

Shameless Plug

https://www.packtpub.com/storm-distributed-real-time-

computation-blueprints/book

Page 113: Hadoop Summit Europe 2014: Apache Storm Architecture

Thank You!

Contributions welcome.

Join the storm community at:http://storm.incubator.apache.org

P. Taylor Goetz [email protected] @ptgoetz