SlideShare a Scribd company logo
Future of Apache Storm
Hadoop Summit 2016, Melbourne
2 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
About us
Arun Iyer, Hortonworks
Apache Storm Committer/PMC
arunm@apache.org
Satish Duggana, Hortonworks
Apache Storm Committer/PMC
satishd@apache.org
3 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
4 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Apache Storm 1.0
Maturity and Improved Performance
Release Date: April 12, 2016
5 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Distributed Cache API
6 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Distributed Cache API
 Topology resources
– Dictionaries, ML Models, Geolocation Data, etc.
 Typically packaged in topology jar
– Fine for small files
– Large files negatively impact topology startup time
– Immutable: Changes require repackaging and deployment
7 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Distributed Cache API
 Allows sharing of files (BLOBs) among topologies
 Files can be updated from the command line
 Allows for files from several KB to several GB in size
 Files can change over the lifetime of the topology
 Allows for compression (e.g. zip, tar, gzip)
8 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Distributed Cache API
 Two implementations: LocalFsBlobStore and HdfsBlobStore
 Local implementation supports Replication Factor (not needed for HDFS-backed
implementation)
 Both support ACLs
9 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Distributed Cache API
Creating a blob:
– storm blobstore create --file dict.txt --acl o::rwa --repl-fctr 2 key1
Making it available to a topology:
– storm jar topo.jar my.topo.Class test_topo -c
topology.blobstore.map=‘{"key1":{"localname":"dict.txt",
"uncompress":"false"}}'
10 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Highly Available Nimbus
11 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus
Nimbus HA
ZooKeeper
Supervisor
Worker*
Supervisor
Worker*
Supervisor
Worker*
12 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus
Leader
Nimbus
Nimbus
Nimbus HA
ZooKeeper
Supervisor
Worker*
Supervisor
Worker*
Supervisor
Worker*
13 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus
Leader
Nimbus
Nimbus
Nimbus HA
ZooKeeper
Supervisor
Worker*
Supervisor
Worker*
Supervisor
Worker*
Leader election
14 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus
Nimbus
Leader
Nimbus
Nimbus HA
ZooKeeper
Supervisor
Worker*
Supervisor
Worker*
Supervisor
Worker*
15 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus
Nimbus
Leader
Nimbus
Nimbus HA
ZooKeeper
Supervisor
Worker*
Supervisor
Worker*
Supervisor
Worker*
16 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Nimbus HA
 Increase overall availability of Nimbus
 Nimbus hosts can join/leave at any time
 Leverages Distributed Cache API
 Topology JAR, Config, and Serialized Topology uploaded to Distributed Cache
 Replication guarantees availability of all files
17 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Pacemaker
Heartbeat Server
18 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Pacemaker
 Replaces Zookeeper for Heartbeats
 In-Memory key-value store
 Allows Scaling to 2k-3k+ Nodes
 Secure: Kerberos/Digest Authentication
19 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Pacemaker
 Compared to Zookeeper
– Less Memory/CPU
– No Disk
– No overhead of maintaining consistency
20 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Automatic Back Pressure
21 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Automatic Back Pressure
 In previous Storm versions, the only way to throttle topologies was
to enable ACKing and set topology.spout.max.pending.
 If you don’t require at-least-once guarantees, this imposed a
significant performance penalty.
22 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Automatic Back Pressure
 High/Low Watermarks (expressed as % of task’s buffer size)
 Back Pressure thread monitors task’s buffers
 If High Watermark reached, slow down Spouts
 If Low Watermark reached, stop throttling
 All Spouts Supported
23 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Performance
24 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Up to 16x faster throughput
Realistically 3x -- Highly dependent on use case and fault tolerance settings
25 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
> 60% Latency Reduction
26 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Resource Aware Scheduler
(RAS)
27 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Resource Aware Scheduler
 Specify the resource requirements (Memory/CPU) for individual
topology components (Spouts/Bolts)
 Memory: On-Heap / Off-Heap (if off-heap is used)
 CPU: Point system based on number of cores
 Resource requirements are per component instance (parallelism
matters)
28 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Resource Aware Scheduler
 CPU and Memory availability described in storm.yaml on each
supervisor node. E.g.:
supervisor.memory.capacity.mb: 3072.0
supervisor.cpu.capacity: 400.0
 Convention for CPU capacity is to use 100 for each CPU core
29 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Resource Aware Scheduler
SpoutDeclarer spout = builder.setSpout("sp1", new TestSpout(), 10);
//set cpu requirement
spout.setCPULoad(20);
//set onheap and offheap memory requirement
spout.setMemoryLoad(64, 16);
BoltDeclarer bolt1 = builder.setBolt("b1", new MyBolt(), 3).shuffleGrouping("sp1");
//sets cpu requirement. Not neccessary to set both CPU and memory.
//For requirements not set, a default value will be used
bolt1.setCPULoad(15);
BoltDeclarer bolt2 = builder.setBolt("b2", new MyBolt(), 2).shuffleGrouping("b1");
bolt2.setMemoryLoad(100);
30 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
31 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
 Fundamental abstraction for processing continuous streams of events
 Breaks the stream of events into sub-sets
 The sub-sets can then be aggregated, joined or analyzed.
Why is it important?
32 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
 Trending items
– The top trending search queries in the last day
– Top tweets in the last 1 hour
 Complex event processing
– Windowing is fundamental to gain meaningful insights from a stream of events
– E.g. If same credit card is used from geographically distributed locations within the last one hour, it
could be a fraud.
Some use cases
33 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
Sliding Window
time0 5 10 15
time0 5 10 15
Tumbling Window
Windows do not overlap
Windows can overlap
34 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
public class WindowMaxBolt extends BaseWindowedBolt {
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void execute(TupleWindow inputWindow) {
int max = Integer.MIN_VALUE;
for (Tuple tuple: inputWindow.get()) {
max = Math.Max(max, tuple.getIntegerByField(”val"));
}
collector.emit(new Values(max));
}
public static void main(String[] args) {
// ...
builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2)));
// submit topology
StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
}
}
35 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
public class WindowMaxBolt extends BaseWindowedBolt {
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void execute(TupleWindow inputWindow) {
int max = Integer.MIN_VALUE;
for (Tuple tuple: inputWindow.get()) {
max = Math.Max(max, tuple.getIntegerByField(”val"));
}
collector.emit(new Values(max));
}
public static void main(String[] args) {
// ...
builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2)));
// submit topology
StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
}
}
triggered when the window activates
based on the window configuration
36 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
public class WindowMaxBolt extends BaseWindowedBolt {
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void execute(TupleWindow inputWindow) {
int max = Integer.MIN_VALUE;
for (Tuple tuple: inputWindow.get()) {
max = Math.Max(max, tuple.getIntegerByField(”val"));
}
collector.emit(new Values(max));
}
public static void main(String[] args) {
// ...
builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2)));
// submit topology
StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
}
}
the window configuration
37 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
Trident
topology.newStream("spout", spout)
.window(
SlidingDurationWindow.of(Duration.minutes(10), Duration.minutes(2)), // window configuration
new InMemoryWindowsStoreFactory(), // window store factory
new Fields("val"), // input field
new Min("val"), // aggregate function
new Fields("max") // output field
);
38 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Native windowing support
 Processing time
 Event time
– e.g. Processing events out of a log file based on when the event actually happened
– Implemented based on watermarks
 Out of order and late events
– e.g. Mobile gaming data where the user goes offline and data arrives after sometime
Windowing considerations
6:026:01 6:05 6:08
7:017:00
6:10 6:12
7:02
6:15
Processing time
6:09 6:126:08
watermark watermark
6:10
late event
39 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
Stateful Bolts with Automatic Checkpointing
40 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
 Allows storm bolts to store and retrieve the state of its computation
 The framework automatically and periodically snapshots the state of the bolts
 In-memory and Redis based state stores
 Implementation based on
– Asynchronous Barrier Snapshotting (ABS) algorithm [1]
– Chandy-Lamport Algorithm [2]
1. http://arxiv.org/abs/1506.08603
2. http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf
41 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
public class WordCountBolt extends BaseStatefulBolt<KeyValueState> {
private KeyValueState wordCounts;
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void initState(KeyValueState state) {
this.wordCounts = state;
}
public void execute(Tuple tuple) {
String word = tuple.getString(0);
Integer count = (Integer) wordCounts.get(word, 0);
count++;
wordCounts.put(word, count);
collector.emit(new Values(word, count));
}
}
42 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
public class WordCountBolt extends BaseStatefulBolt<KeyValueState> {
private KeyValueState wordCounts;
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void initState(KeyValueState state) {
this.wordCounts = state;
}
public void execute(Tuple tuple) {
String word = tuple.getString(0);
Integer count = (Integer) wordCounts.get(word, 0);
count++;
wordCounts.put(word, count);
collector.emit(new Values(word, count));
}
}
Initialize state
43 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
public class WordCountBolt extends BaseStatefulBolt<KeyValueState> {
private KeyValueState wordCounts;
private OutputCollector collector;
public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
public void initState(KeyValueState state) {
this.wordCounts = state;
}
public void execute(Tuple tuple) {
String word = tuple.getString(0);
Integer count = (Integer) wordCounts.get(word, 0);
count++;
wordCounts.put(word, count);
collector.emit(new Values(word, count));
}
}
Update state
44 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
 Checkpointing/Snapshotting: What you see
Spout Stateful Bolt1 Stateful Bolt2Bolt1
execute/update state execute/update stateexecute
45 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
State Management
 Checkpointing/Snapshotting: What you get
Spout
Checkpoint Spout
Stateful Bolt1 Stateful Bolt2Bolt1
State Store
ACKER
ACK
ACKACK
chkpt chkpt
chkpt
46 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
(currently Beta/WIP)
47 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
 SQL is a well adopted standard
– available in several projects like Drill, Hive, Phoenix, Spark
 Faster development cycle
 Opportunity to unify batch and stream data analytics
Benefits
SELECT
word, COUNT(*) AS total
FROM
words
GROUP BY word
48 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
 Leverages Apache calcite
 Compiles SQL statements into Trident topologies
 Usage:
– $ bin/storm sql <sql-file> <topo-name>
Storm implementation
49 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
 Example
CREATE EXTERNAL TABLE ORDERS
(ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT)
LOCATION 'kafka://localhost:2181/brokers?topic=orders’
CREATE EXTERNAL TABLE LARGE_ORDERS
(ID INT PRIMARY KEY, TOTAL INT)
LOCATION 'kafka://localhost:2181/brokers?topic=large_orders’
INSERT INTO LARGE_ORDERS
SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL
FROM ORDERS
WHERE UNIT_PRICE * QUANTITY > 50
50 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
 Storm SQL workflow
51 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Streaming SQL
 Storm SQL workflow
52 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Storm Usability Improvements
Enhanced Debugging and Monitoring of Topologies
53 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Storm usability improvements
./bin/storm set_log_level [topology name]-l [logger name]=[LEVEL]:[TIMEOUT]
Dynamic Log Levels
Storm CLI:
Storm UI:
54 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Storm usability improvements
 No more debug bolts or Trident functions
 Click on the “Events” link to view the sample log.
Tuple Sampling
55 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Storm usability improvements
 Distributed Log search
– Search across all topology logs, even archived (ZIP) logs.
 Dynamic worker profiling
– Heap dumps, JStack output, Profiler recordings
 Restart workers from UI
 Supervisor health checks
56 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Integrations
New
 Cassandra
 Solr
 Elastic Search
 MQTT
Improvements
 Kafka
 HDFS spout
 Hbase
 Hive
57 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
What’s next for Storm?
2.0 and Beyond
58 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Clojure to Java
Broadening the contributor base
Alibaba JStorm Contribution
59 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Apache Beam (incubating) Integration
 Unified API for dealing with
bounded/unbounded data sources (i.e.
batch/streaming)
 One API. Multiple implementations
(execution engines). Called “Runners” in
Beamspeak.
60 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Apache Storm 2.0 and beyond
 Windowing enhancements
 Unified Stream API
 Revamped UI
 Metrics improvements
and many more…
61 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
Q&A
Thank You

More Related Content

What's hot

Connecting the Drops with Apache NiFi & Apache MiNiFi
Connecting the Drops with Apache NiFi & Apache MiNiFiConnecting the Drops with Apache NiFi & Apache MiNiFi
Connecting the Drops with Apache NiFi & Apache MiNiFi
DataWorks Summit
 
Apache Hive 2.0: SQL, Speed, Scale
Apache Hive 2.0: SQL, Speed, ScaleApache Hive 2.0: SQL, Speed, Scale
Apache Hive 2.0: SQL, Speed, Scale
DataWorks Summit/Hadoop Summit
 
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
StampedeCon
 
Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem
DataWorks Summit/Hadoop Summit
 
Tracing your security telemetry with Apache Metron
Tracing your security telemetry with Apache MetronTracing your security telemetry with Apache Metron
Tracing your security telemetry with Apache Metron
DataWorks Summit/Hadoop Summit
 
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on HiveFaster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
DataWorks Summit/Hadoop Summit
 
A Multi Colored YARN
A Multi Colored YARNA Multi Colored YARN
A Multi Colored YARN
DataWorks Summit/Hadoop Summit
 
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
DataWorks Summit
 
LLAP: Building Cloud First BI
LLAP: Building Cloud First BILLAP: Building Cloud First BI
LLAP: Building Cloud First BI
DataWorks Summit
 
Running a container cloud on YARN
Running a container cloud on YARNRunning a container cloud on YARN
Running a container cloud on YARN
DataWorks Summit
 
Apache NiFi 1.0 in Nutshell
Apache NiFi 1.0 in NutshellApache NiFi 1.0 in Nutshell
Apache NiFi 1.0 in Nutshell
DataWorks Summit/Hadoop Summit
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
DataWorks Summit
 
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
Cedric CARBONE
 
An Overview on Optimization in Apache Hive: Past, Present Future
An Overview on Optimization in Apache Hive: Past, Present FutureAn Overview on Optimization in Apache Hive: Past, Present Future
An Overview on Optimization in Apache Hive: Past, Present Future
DataWorks Summit/Hadoop Summit
 
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
alanfgates
 
Apache Eagle: Architecture Evolvement and New Features
Apache Eagle: Architecture Evolvement and New FeaturesApache Eagle: Architecture Evolvement and New Features
Apache Eagle: Architecture Evolvement and New Features
Hao Chen
 
Enabling Diverse Workload Scheduling in YARN
Enabling Diverse Workload Scheduling in YARNEnabling Diverse Workload Scheduling in YARN
Enabling Diverse Workload Scheduling in YARN
DataWorks Summit
 
Network for the Large-scale Hadoop cluster at Yahoo! JAPAN
Network for the Large-scale Hadoop cluster at Yahoo! JAPANNetwork for the Large-scale Hadoop cluster at Yahoo! JAPAN
Network for the Large-scale Hadoop cluster at Yahoo! JAPAN
DataWorks Summit/Hadoop Summit
 
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
Mich Talebzadeh (Ph.D.)
 
Apache Eagle in Action
Apache Eagle in ActionApache Eagle in Action
Apache Eagle in Action
Hao Chen
 

What's hot (20)

Connecting the Drops with Apache NiFi & Apache MiNiFi
Connecting the Drops with Apache NiFi & Apache MiNiFiConnecting the Drops with Apache NiFi & Apache MiNiFi
Connecting the Drops with Apache NiFi & Apache MiNiFi
 
Apache Hive 2.0: SQL, Speed, Scale
Apache Hive 2.0: SQL, Speed, ScaleApache Hive 2.0: SQL, Speed, Scale
Apache Hive 2.0: SQL, Speed, Scale
 
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
Apache Hadoop YARN – Multi-Tenancy, Capacity Scheduler & Preemption - Stamped...
 
Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem
 
Tracing your security telemetry with Apache Metron
Tracing your security telemetry with Apache MetronTracing your security telemetry with Apache Metron
Tracing your security telemetry with Apache Metron
 
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on HiveFaster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
Faster, Faster, Faster: The True Story of a Mobile Analytics Data Mart on Hive
 
A Multi Colored YARN
A Multi Colored YARNA Multi Colored YARN
A Multi Colored YARN
 
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
Real-Time Ingesting and Transforming Sensor Data and Social Data with NiFi an...
 
LLAP: Building Cloud First BI
LLAP: Building Cloud First BILLAP: Building Cloud First BI
LLAP: Building Cloud First BI
 
Running a container cloud on YARN
Running a container cloud on YARNRunning a container cloud on YARN
Running a container cloud on YARN
 
Apache NiFi 1.0 in Nutshell
Apache NiFi 1.0 in NutshellApache NiFi 1.0 in Nutshell
Apache NiFi 1.0 in Nutshell
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
 
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
Apache Falcon : 22 Sept 2014 for Hadoop User Group France (@Criteo)
 
An Overview on Optimization in Apache Hive: Past, Present Future
An Overview on Optimization in Apache Hive: Past, Present FutureAn Overview on Optimization in Apache Hive: Past, Present Future
An Overview on Optimization in Apache Hive: Past, Present Future
 
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
Hive2.0 sql speed-scale--hadoop-summit-dublin-apr-2016
 
Apache Eagle: Architecture Evolvement and New Features
Apache Eagle: Architecture Evolvement and New FeaturesApache Eagle: Architecture Evolvement and New Features
Apache Eagle: Architecture Evolvement and New Features
 
Enabling Diverse Workload Scheduling in YARN
Enabling Diverse Workload Scheduling in YARNEnabling Diverse Workload Scheduling in YARN
Enabling Diverse Workload Scheduling in YARN
 
Network for the Large-scale Hadoop cluster at Yahoo! JAPAN
Network for the Large-scale Hadoop cluster at Yahoo! JAPANNetwork for the Large-scale Hadoop cluster at Yahoo! JAPAN
Network for the Large-scale Hadoop cluster at Yahoo! JAPAN
 
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
Query Engines for Hive: MR, Spark, Tez with LLAP – Considerations!
 
Apache Eagle in Action
Apache Eagle in ActionApache Eagle in Action
Apache Eagle in Action
 

Viewers also liked

The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
P. Taylor Goetz
 
Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014
P. Taylor Goetz
 
Apache Storm 0.9 basic training - Verisign
Apache Storm 0.9 basic training - VerisignApache Storm 0.9 basic training - Verisign
Apache Storm 0.9 basic training - Verisign
Michael Noll
 
Creating the Internet of Your Things
Creating the Internet of Your ThingsCreating the Internet of Your Things
Creating the Internet of Your Things
DataWorks Summit/Hadoop Summit
 
Storm
StormStorm
Storm
nathanmarz
 
Storms!
Storms!Storms!
Spark Streaming and Expert Systems
Spark Streaming and Expert SystemsSpark Streaming and Expert Systems
Spark Streaming and Expert Systems
Jim Haughwout
 
Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)
Robert Evans
 
So we're running Apache ZooKeeper. Now What? By Camille Fournier
So we're running Apache ZooKeeper. Now What? By Camille Fournier So we're running Apache ZooKeeper. Now What? By Camille Fournier
So we're running Apache ZooKeeper. Now What? By Camille Fournier
Hakka Labs
 
Spark Streaming the Industrial IoT
Spark Streaming the Industrial IoTSpark Streaming the Industrial IoT
Spark Streaming the Industrial IoT
Jim Haughwout
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
Chicago Hadoop Users Group
 
TensorFrames: Google Tensorflow on Apache Spark
TensorFrames: Google Tensorflow on Apache SparkTensorFrames: Google Tensorflow on Apache Spark
TensorFrames: Google Tensorflow on Apache Spark
Databricks
 
Hadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm ArchitectureHadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm Architecture
P. Taylor Goetz
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark Streaming
P. Taylor Goetz
 
Kafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtimeKafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtime
Guido Schmutz
 

Viewers also liked (15)

The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
 
Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014
 
Apache Storm 0.9 basic training - Verisign
Apache Storm 0.9 basic training - VerisignApache Storm 0.9 basic training - Verisign
Apache Storm 0.9 basic training - Verisign
 
Creating the Internet of Your Things
Creating the Internet of Your ThingsCreating the Internet of Your Things
Creating the Internet of Your Things
 
Storm
StormStorm
Storm
 
Storms!
Storms!Storms!
Storms!
 
Spark Streaming and Expert Systems
Spark Streaming and Expert SystemsSpark Streaming and Expert Systems
Spark Streaming and Expert Systems
 
Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)
 
So we're running Apache ZooKeeper. Now What? By Camille Fournier
So we're running Apache ZooKeeper. Now What? By Camille Fournier So we're running Apache ZooKeeper. Now What? By Camille Fournier
So we're running Apache ZooKeeper. Now What? By Camille Fournier
 
Spark Streaming the Industrial IoT
Spark Streaming the Industrial IoTSpark Streaming the Industrial IoT
Spark Streaming the Industrial IoT
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 
TensorFrames: Google Tensorflow on Apache Spark
TensorFrames: Google Tensorflow on Apache SparkTensorFrames: Google Tensorflow on Apache Spark
TensorFrames: Google Tensorflow on Apache Spark
 
Hadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm ArchitectureHadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm Architecture
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark Streaming
 
Kafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtimeKafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtime
 

Similar to Future of Apache Storm

Apache Ambari: Past, Present, Future
Apache Ambari: Past, Present, FutureApache Ambari: Past, Present, Future
Apache Ambari: Past, Present, Future
Hortonworks
 
Taming the Elephant: Efficient and Effective Apache Hadoop Management
Taming the Elephant: Efficient and Effective Apache Hadoop ManagementTaming the Elephant: Efficient and Effective Apache Hadoop Management
Taming the Elephant: Efficient and Effective Apache Hadoop Management
DataWorks Summit/Hadoop Summit
 
IoT:what about data storage?
IoT:what about data storage?IoT:what about data storage?
IoT:what about data storage?
DataWorks Summit/Hadoop Summit
 
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
Aravindan Vijayan
 
ACID Transactions in Hive
ACID Transactions in HiveACID Transactions in Hive
ACID Transactions in Hive
Eugene Koifman
 
Druid Scaling Realtime Analytics
Druid Scaling Realtime AnalyticsDruid Scaling Realtime Analytics
Druid Scaling Realtime Analytics
Aaron Brooks
 
Spark Summit EU talk by Steve Loughran
Spark Summit EU talk by Steve LoughranSpark Summit EU talk by Steve Loughran
Spark Summit EU talk by Steve Loughran
Spark Summit
 
Put is the new rename: San Jose Summit Edition
Put is the new rename: San Jose Summit EditionPut is the new rename: San Jose Summit Edition
Put is the new rename: San Jose Summit Edition
Steve Loughran
 
Streamline - Stream Analytics for Everyone
Streamline - Stream Analytics for EveryoneStreamline - Stream Analytics for Everyone
Streamline - Stream Analytics for Everyone
DataWorks Summit/Hadoop Summit
 
Running Cloudbreak on Kubernetes
Running Cloudbreak on KubernetesRunning Cloudbreak on Kubernetes
Running Cloudbreak on Kubernetes
Krisztián Horváth
 
Running Cloudbreak on Kubernetes
Running Cloudbreak on KubernetesRunning Cloudbreak on Kubernetes
Running Cloudbreak on Kubernetes
Future of Data Meetup
 
Storm – Streaming Data Analytics at Scale - StampedeCon 2014
Storm – Streaming Data Analytics at Scale - StampedeCon 2014Storm – Streaming Data Analytics at Scale - StampedeCon 2014
Storm – Streaming Data Analytics at Scale - StampedeCon 2014
StampedeCon
 
Druid deep dive
Druid deep diveDruid deep dive
Druid deep dive
Kashif Khan
 
Hortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
Hortonworks Technical Workshop: Real Time Monitoring with Apache HadoopHortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
Hortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
Hortonworks
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
DataWorks Summit/Hadoop Summit
 
Dataworks Berlin Summit 18' - Deep learning On YARN - Running Distributed Te...
Dataworks Berlin Summit 18' - Deep learning On YARN -  Running Distributed Te...Dataworks Berlin Summit 18' - Deep learning On YARN -  Running Distributed Te...
Dataworks Berlin Summit 18' - Deep learning On YARN - Running Distributed Te...
Wangda Tan
 
Ambari Management Packs (Apache Ambari Meetup 2018)
Ambari Management Packs (Apache Ambari Meetup 2018)Ambari Management Packs (Apache Ambari Meetup 2018)
Ambari Management Packs (Apache Ambari Meetup 2018)
Swapan Shridhar
 
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
Hortonworks
 
Apache Spark and Object Stores
Apache Spark and Object StoresApache Spark and Object Stores
Apache Spark and Object Stores
Steve Loughran
 
Mhug apache storm
Mhug apache stormMhug apache storm
Mhug apache storm
Joseph Niemiec
 

Similar to Future of Apache Storm (20)

Apache Ambari: Past, Present, Future
Apache Ambari: Past, Present, FutureApache Ambari: Past, Present, Future
Apache Ambari: Past, Present, Future
 
Taming the Elephant: Efficient and Effective Apache Hadoop Management
Taming the Elephant: Efficient and Effective Apache Hadoop ManagementTaming the Elephant: Efficient and Effective Apache Hadoop Management
Taming the Elephant: Efficient and Effective Apache Hadoop Management
 
IoT:what about data storage?
IoT:what about data storage?IoT:what about data storage?
IoT:what about data storage?
 
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
Ambari metrics system - Apache ambari meetup (DataWorks Summit 2017)
 
ACID Transactions in Hive
ACID Transactions in HiveACID Transactions in Hive
ACID Transactions in Hive
 
Druid Scaling Realtime Analytics
Druid Scaling Realtime AnalyticsDruid Scaling Realtime Analytics
Druid Scaling Realtime Analytics
 
Spark Summit EU talk by Steve Loughran
Spark Summit EU talk by Steve LoughranSpark Summit EU talk by Steve Loughran
Spark Summit EU talk by Steve Loughran
 
Put is the new rename: San Jose Summit Edition
Put is the new rename: San Jose Summit EditionPut is the new rename: San Jose Summit Edition
Put is the new rename: San Jose Summit Edition
 
Streamline - Stream Analytics for Everyone
Streamline - Stream Analytics for EveryoneStreamline - Stream Analytics for Everyone
Streamline - Stream Analytics for Everyone
 
Running Cloudbreak on Kubernetes
Running Cloudbreak on KubernetesRunning Cloudbreak on Kubernetes
Running Cloudbreak on Kubernetes
 
Running Cloudbreak on Kubernetes
Running Cloudbreak on KubernetesRunning Cloudbreak on Kubernetes
Running Cloudbreak on Kubernetes
 
Storm – Streaming Data Analytics at Scale - StampedeCon 2014
Storm – Streaming Data Analytics at Scale - StampedeCon 2014Storm – Streaming Data Analytics at Scale - StampedeCon 2014
Storm – Streaming Data Analytics at Scale - StampedeCon 2014
 
Druid deep dive
Druid deep diveDruid deep dive
Druid deep dive
 
Hortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
Hortonworks Technical Workshop: Real Time Monitoring with Apache HadoopHortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
Hortonworks Technical Workshop: Real Time Monitoring with Apache Hadoop
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
 
Dataworks Berlin Summit 18' - Deep learning On YARN - Running Distributed Te...
Dataworks Berlin Summit 18' - Deep learning On YARN -  Running Distributed Te...Dataworks Berlin Summit 18' - Deep learning On YARN -  Running Distributed Te...
Dataworks Berlin Summit 18' - Deep learning On YARN - Running Distributed Te...
 
Ambari Management Packs (Apache Ambari Meetup 2018)
Ambari Management Packs (Apache Ambari Meetup 2018)Ambari Management Packs (Apache Ambari Meetup 2018)
Ambari Management Packs (Apache Ambari Meetup 2018)
 
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
Hortonworks Data in Motion Webinar Series Part 7 Apache Kafka Nifi Better Tog...
 
Apache Spark and Object Stores
Apache Spark and Object StoresApache Spark and Object Stores
Apache Spark and Object Stores
 
Mhug apache storm
Mhug apache stormMhug apache storm
Mhug apache storm
 

More from DataWorks Summit/Hadoop Summit

Running Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in ProductionRunning Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in Production
DataWorks Summit/Hadoop Summit
 
State of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache ZeppelinState of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache Zeppelin
DataWorks Summit/Hadoop Summit
 
Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache Ranger
DataWorks Summit/Hadoop Summit
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science Platform
DataWorks Summit/Hadoop Summit
 
Revolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and ZeppelinRevolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and Zeppelin
DataWorks Summit/Hadoop Summit
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSense
DataWorks Summit/Hadoop Summit
 
Hadoop Crash Course
Hadoop Crash CourseHadoop Crash Course
Hadoop Crash Course
DataWorks Summit/Hadoop Summit
 
Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
DataWorks Summit/Hadoop Summit
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
DataWorks Summit/Hadoop Summit
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
DataWorks Summit/Hadoop Summit
 
Schema Registry - Set you Data Free
Schema Registry - Set you Data FreeSchema Registry - Set you Data Free
Schema Registry - Set you Data Free
DataWorks Summit/Hadoop Summit
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
DataWorks Summit/Hadoop Summit
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
DataWorks Summit/Hadoop Summit
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and ML
DataWorks Summit/Hadoop Summit
 
How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient
DataWorks Summit/Hadoop Summit
 
HBase in Practice
HBase in Practice HBase in Practice
HBase in Practice
DataWorks Summit/Hadoop Summit
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)
DataWorks Summit/Hadoop Summit
 
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS HadoopBreaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
DataWorks Summit/Hadoop Summit
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
DataWorks Summit/Hadoop Summit
 
Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop
DataWorks Summit/Hadoop Summit
 

More from DataWorks Summit/Hadoop Summit (20)

Running Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in ProductionRunning Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in Production
 
State of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache ZeppelinState of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache Zeppelin
 
Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache Ranger
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science Platform
 
Revolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and ZeppelinRevolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and Zeppelin
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSense
 
Hadoop Crash Course
Hadoop Crash CourseHadoop Crash Course
Hadoop Crash Course
 
Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
 
Schema Registry - Set you Data Free
Schema Registry - Set you Data FreeSchema Registry - Set you Data Free
Schema Registry - Set you Data Free
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and ML
 
How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient
 
HBase in Practice
HBase in Practice HBase in Practice
HBase in Practice
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)
 
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS HadoopBreaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
 
Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop
 

Recently uploaded

Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with SlackLet's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
shyamraj55
 
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
alexjohnson7307
 
Skybuffer SAM4U tool for SAP license adoption
Skybuffer SAM4U tool for SAP license adoptionSkybuffer SAM4U tool for SAP license adoption
Skybuffer SAM4U tool for SAP license adoption
Tatiana Kojar
 
WeTestAthens: Postman's AI & Automation Techniques
WeTestAthens: Postman's AI & Automation TechniquesWeTestAthens: Postman's AI & Automation Techniques
WeTestAthens: Postman's AI & Automation Techniques
Postman
 
Energy Efficient Video Encoding for Cloud and Edge Computing Instances
Energy Efficient Video Encoding for Cloud and Edge Computing InstancesEnergy Efficient Video Encoding for Cloud and Edge Computing Instances
Energy Efficient Video Encoding for Cloud and Edge Computing Instances
Alpen-Adria-Universität
 
Introduction of Cybersecurity with OSS at Code Europe 2024
Introduction of Cybersecurity with OSS  at Code Europe 2024Introduction of Cybersecurity with OSS  at Code Europe 2024
Introduction of Cybersecurity with OSS at Code Europe 2024
Hiroshi SHIBATA
 
Building Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and MilvusBuilding Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and Milvus
Zilliz
 
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
saastr
 
Nordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptxNordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptx
MichaelKnudsen27
 
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
Edge AI and Vision Alliance
 
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
Tatiana Kojar
 
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
Alex Pruden
 
June Patch Tuesday
June Patch TuesdayJune Patch Tuesday
June Patch Tuesday
Ivanti
 
SAP S/4 HANA sourcing and procurement to Public cloud
SAP S/4 HANA sourcing and procurement to Public cloudSAP S/4 HANA sourcing and procurement to Public cloud
SAP S/4 HANA sourcing and procurement to Public cloud
maazsz111
 
Fueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte WebinarFueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte Webinar
Zilliz
 
Choosing The Best AWS Service For Your Website + API.pptx
Choosing The Best AWS Service For Your Website + API.pptxChoosing The Best AWS Service For Your Website + API.pptx
Choosing The Best AWS Service For Your Website + API.pptx
Brandon Minnick, MBA
 
dbms calicut university B. sc Cs 4th sem.pdf
dbms  calicut university B. sc Cs 4th sem.pdfdbms  calicut university B. sc Cs 4th sem.pdf
dbms calicut university B. sc Cs 4th sem.pdf
Shinana2
 
Digital Marketing Trends in 2024 | Guide for Staying Ahead
Digital Marketing Trends in 2024 | Guide for Staying AheadDigital Marketing Trends in 2024 | Guide for Staying Ahead
Digital Marketing Trends in 2024 | Guide for Staying Ahead
Wask
 
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
Jeffrey Haguewood
 
Taking AI to the Next Level in Manufacturing.pdf
Taking AI to the Next Level in Manufacturing.pdfTaking AI to the Next Level in Manufacturing.pdf
Taking AI to the Next Level in Manufacturing.pdf
ssuserfac0301
 

Recently uploaded (20)

Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with SlackLet's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
Let's Integrate MuleSoft RPA, COMPOSER, APM with AWS IDP along with Slack
 
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
leewayhertz.com-AI in predictive maintenance Use cases technologies benefits ...
 
Skybuffer SAM4U tool for SAP license adoption
Skybuffer SAM4U tool for SAP license adoptionSkybuffer SAM4U tool for SAP license adoption
Skybuffer SAM4U tool for SAP license adoption
 
WeTestAthens: Postman's AI & Automation Techniques
WeTestAthens: Postman's AI & Automation TechniquesWeTestAthens: Postman's AI & Automation Techniques
WeTestAthens: Postman's AI & Automation Techniques
 
Energy Efficient Video Encoding for Cloud and Edge Computing Instances
Energy Efficient Video Encoding for Cloud and Edge Computing InstancesEnergy Efficient Video Encoding for Cloud and Edge Computing Instances
Energy Efficient Video Encoding for Cloud and Edge Computing Instances
 
Introduction of Cybersecurity with OSS at Code Europe 2024
Introduction of Cybersecurity with OSS  at Code Europe 2024Introduction of Cybersecurity with OSS  at Code Europe 2024
Introduction of Cybersecurity with OSS at Code Europe 2024
 
Building Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and MilvusBuilding Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and Milvus
 
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
Deep Dive: AI-Powered Marketing to Get More Leads and Customers with HyperGro...
 
Nordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptxNordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptx
 
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
 
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
Skybuffer AI: Advanced Conversational and Generative AI Solution on SAP Busin...
 
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
 
June Patch Tuesday
June Patch TuesdayJune Patch Tuesday
June Patch Tuesday
 
SAP S/4 HANA sourcing and procurement to Public cloud
SAP S/4 HANA sourcing and procurement to Public cloudSAP S/4 HANA sourcing and procurement to Public cloud
SAP S/4 HANA sourcing and procurement to Public cloud
 
Fueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte WebinarFueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte Webinar
 
Choosing The Best AWS Service For Your Website + API.pptx
Choosing The Best AWS Service For Your Website + API.pptxChoosing The Best AWS Service For Your Website + API.pptx
Choosing The Best AWS Service For Your Website + API.pptx
 
dbms calicut university B. sc Cs 4th sem.pdf
dbms  calicut university B. sc Cs 4th sem.pdfdbms  calicut university B. sc Cs 4th sem.pdf
dbms calicut university B. sc Cs 4th sem.pdf
 
Digital Marketing Trends in 2024 | Guide for Staying Ahead
Digital Marketing Trends in 2024 | Guide for Staying AheadDigital Marketing Trends in 2024 | Guide for Staying Ahead
Digital Marketing Trends in 2024 | Guide for Staying Ahead
 
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
Salesforce Integration for Bonterra Impact Management (fka Social Solutions A...
 
Taking AI to the Next Level in Manufacturing.pdf
Taking AI to the Next Level in Manufacturing.pdfTaking AI to the Next Level in Manufacturing.pdf
Taking AI to the Next Level in Manufacturing.pdf
 

Future of Apache Storm

  • 1. Future of Apache Storm Hadoop Summit 2016, Melbourne
  • 2. 2 © Hortonworks Inc. 2011 – 2016. All Rights Reserved About us Arun Iyer, Hortonworks Apache Storm Committer/PMC arunm@apache.org Satish Duggana, Hortonworks Apache Storm Committer/PMC satishd@apache.org
  • 3. 3 © Hortonworks Inc. 2011 – 2016. All Rights Reserved
  • 4. 4 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Apache Storm 1.0 Maturity and Improved Performance Release Date: April 12, 2016
  • 5. 5 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Distributed Cache API
  • 6. 6 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Distributed Cache API  Topology resources – Dictionaries, ML Models, Geolocation Data, etc.  Typically packaged in topology jar – Fine for small files – Large files negatively impact topology startup time – Immutable: Changes require repackaging and deployment
  • 7. 7 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Distributed Cache API  Allows sharing of files (BLOBs) among topologies  Files can be updated from the command line  Allows for files from several KB to several GB in size  Files can change over the lifetime of the topology  Allows for compression (e.g. zip, tar, gzip)
  • 8. 8 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Distributed Cache API  Two implementations: LocalFsBlobStore and HdfsBlobStore  Local implementation supports Replication Factor (not needed for HDFS-backed implementation)  Both support ACLs
  • 9. 9 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Distributed Cache API Creating a blob: – storm blobstore create --file dict.txt --acl o::rwa --repl-fctr 2 key1 Making it available to a topology: – storm jar topo.jar my.topo.Class test_topo -c topology.blobstore.map=‘{"key1":{"localname":"dict.txt", "uncompress":"false"}}'
  • 10. 10 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Highly Available Nimbus
  • 11. 11 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus Nimbus HA ZooKeeper Supervisor Worker* Supervisor Worker* Supervisor Worker*
  • 12. 12 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus Leader Nimbus Nimbus Nimbus HA ZooKeeper Supervisor Worker* Supervisor Worker* Supervisor Worker*
  • 13. 13 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus Leader Nimbus Nimbus Nimbus HA ZooKeeper Supervisor Worker* Supervisor Worker* Supervisor Worker* Leader election
  • 14. 14 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus Nimbus Leader Nimbus Nimbus HA ZooKeeper Supervisor Worker* Supervisor Worker* Supervisor Worker*
  • 15. 15 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus Nimbus Leader Nimbus Nimbus HA ZooKeeper Supervisor Worker* Supervisor Worker* Supervisor Worker*
  • 16. 16 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Nimbus HA  Increase overall availability of Nimbus  Nimbus hosts can join/leave at any time  Leverages Distributed Cache API  Topology JAR, Config, and Serialized Topology uploaded to Distributed Cache  Replication guarantees availability of all files
  • 17. 17 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Pacemaker Heartbeat Server
  • 18. 18 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Pacemaker  Replaces Zookeeper for Heartbeats  In-Memory key-value store  Allows Scaling to 2k-3k+ Nodes  Secure: Kerberos/Digest Authentication
  • 19. 19 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Pacemaker  Compared to Zookeeper – Less Memory/CPU – No Disk – No overhead of maintaining consistency
  • 20. 20 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Automatic Back Pressure
  • 21. 21 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Automatic Back Pressure  In previous Storm versions, the only way to throttle topologies was to enable ACKing and set topology.spout.max.pending.  If you don’t require at-least-once guarantees, this imposed a significant performance penalty.
  • 22. 22 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Automatic Back Pressure  High/Low Watermarks (expressed as % of task’s buffer size)  Back Pressure thread monitors task’s buffers  If High Watermark reached, slow down Spouts  If Low Watermark reached, stop throttling  All Spouts Supported
  • 23. 23 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Performance
  • 24. 24 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Up to 16x faster throughput Realistically 3x -- Highly dependent on use case and fault tolerance settings
  • 25. 25 © Hortonworks Inc. 2011 – 2016. All Rights Reserved > 60% Latency Reduction
  • 26. 26 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Resource Aware Scheduler (RAS)
  • 27. 27 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Resource Aware Scheduler  Specify the resource requirements (Memory/CPU) for individual topology components (Spouts/Bolts)  Memory: On-Heap / Off-Heap (if off-heap is used)  CPU: Point system based on number of cores  Resource requirements are per component instance (parallelism matters)
  • 28. 28 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Resource Aware Scheduler  CPU and Memory availability described in storm.yaml on each supervisor node. E.g.: supervisor.memory.capacity.mb: 3072.0 supervisor.cpu.capacity: 400.0  Convention for CPU capacity is to use 100 for each CPU core
  • 29. 29 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Resource Aware Scheduler SpoutDeclarer spout = builder.setSpout("sp1", new TestSpout(), 10); //set cpu requirement spout.setCPULoad(20); //set onheap and offheap memory requirement spout.setMemoryLoad(64, 16); BoltDeclarer bolt1 = builder.setBolt("b1", new MyBolt(), 3).shuffleGrouping("sp1"); //sets cpu requirement. Not neccessary to set both CPU and memory. //For requirements not set, a default value will be used bolt1.setCPULoad(15); BoltDeclarer bolt2 = builder.setBolt("b2", new MyBolt(), 2).shuffleGrouping("b1"); bolt2.setMemoryLoad(100);
  • 30. 30 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support
  • 31. 31 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support  Fundamental abstraction for processing continuous streams of events  Breaks the stream of events into sub-sets  The sub-sets can then be aggregated, joined or analyzed. Why is it important?
  • 32. 32 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support  Trending items – The top trending search queries in the last day – Top tweets in the last 1 hour  Complex event processing – Windowing is fundamental to gain meaningful insights from a stream of events – E.g. If same credit card is used from geographically distributed locations within the last one hour, it could be a fraud. Some use cases
  • 33. 33 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support Sliding Window time0 5 10 15 time0 5 10 15 Tumbling Window Windows do not overlap Windows can overlap
  • 34. 34 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support public class WindowMaxBolt extends BaseWindowedBolt { private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void execute(TupleWindow inputWindow) { int max = Integer.MIN_VALUE; for (Tuple tuple: inputWindow.get()) { max = Math.Max(max, tuple.getIntegerByField(”val")); } collector.emit(new Values(max)); } public static void main(String[] args) { // ... builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2))); // submit topology StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology()); } }
  • 35. 35 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support public class WindowMaxBolt extends BaseWindowedBolt { private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void execute(TupleWindow inputWindow) { int max = Integer.MIN_VALUE; for (Tuple tuple: inputWindow.get()) { max = Math.Max(max, tuple.getIntegerByField(”val")); } collector.emit(new Values(max)); } public static void main(String[] args) { // ... builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2))); // submit topology StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology()); } } triggered when the window activates based on the window configuration
  • 36. 36 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support public class WindowMaxBolt extends BaseWindowedBolt { private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void execute(TupleWindow inputWindow) { int max = Integer.MIN_VALUE; for (Tuple tuple: inputWindow.get()) { max = Math.Max(max, tuple.getIntegerByField(”val")); } collector.emit(new Values(max)); } public static void main(String[] args) { // ... builder.setBolt("slidingMax", new WindowMaxBolt().withWindow(Duration.minutes(10), Duration.minutes(2))); // submit topology StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology()); } } the window configuration
  • 37. 37 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support Trident topology.newStream("spout", spout) .window( SlidingDurationWindow.of(Duration.minutes(10), Duration.minutes(2)), // window configuration new InMemoryWindowsStoreFactory(), // window store factory new Fields("val"), // input field new Min("val"), // aggregate function new Fields("max") // output field );
  • 38. 38 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Native windowing support  Processing time  Event time – e.g. Processing events out of a log file based on when the event actually happened – Implemented based on watermarks  Out of order and late events – e.g. Mobile gaming data where the user goes offline and data arrives after sometime Windowing considerations 6:026:01 6:05 6:08 7:017:00 6:10 6:12 7:02 6:15 Processing time 6:09 6:126:08 watermark watermark 6:10 late event
  • 39. 39 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management Stateful Bolts with Automatic Checkpointing
  • 40. 40 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management  Allows storm bolts to store and retrieve the state of its computation  The framework automatically and periodically snapshots the state of the bolts  In-memory and Redis based state stores  Implementation based on – Asynchronous Barrier Snapshotting (ABS) algorithm [1] – Chandy-Lamport Algorithm [2] 1. http://arxiv.org/abs/1506.08603 2. http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf
  • 41. 41 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management public class WordCountBolt extends BaseStatefulBolt<KeyValueState> { private KeyValueState wordCounts; private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void initState(KeyValueState state) { this.wordCounts = state; } public void execute(Tuple tuple) { String word = tuple.getString(0); Integer count = (Integer) wordCounts.get(word, 0); count++; wordCounts.put(word, count); collector.emit(new Values(word, count)); } }
  • 42. 42 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management public class WordCountBolt extends BaseStatefulBolt<KeyValueState> { private KeyValueState wordCounts; private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void initState(KeyValueState state) { this.wordCounts = state; } public void execute(Tuple tuple) { String word = tuple.getString(0); Integer count = (Integer) wordCounts.get(word, 0); count++; wordCounts.put(word, count); collector.emit(new Values(word, count)); } } Initialize state
  • 43. 43 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management public class WordCountBolt extends BaseStatefulBolt<KeyValueState> { private KeyValueState wordCounts; private OutputCollector collector; public void prepare(Map conf, TopologyContext context, OutputCollector collector) { this.collector = collector; } public void initState(KeyValueState state) { this.wordCounts = state; } public void execute(Tuple tuple) { String word = tuple.getString(0); Integer count = (Integer) wordCounts.get(word, 0); count++; wordCounts.put(word, count); collector.emit(new Values(word, count)); } } Update state
  • 44. 44 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management  Checkpointing/Snapshotting: What you see Spout Stateful Bolt1 Stateful Bolt2Bolt1 execute/update state execute/update stateexecute
  • 45. 45 © Hortonworks Inc. 2011 – 2016. All Rights Reserved State Management  Checkpointing/Snapshotting: What you get Spout Checkpoint Spout Stateful Bolt1 Stateful Bolt2Bolt1 State Store ACKER ACK ACKACK chkpt chkpt chkpt
  • 46. 46 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL (currently Beta/WIP)
  • 47. 47 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL  SQL is a well adopted standard – available in several projects like Drill, Hive, Phoenix, Spark  Faster development cycle  Opportunity to unify batch and stream data analytics Benefits SELECT word, COUNT(*) AS total FROM words GROUP BY word
  • 48. 48 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL  Leverages Apache calcite  Compiles SQL statements into Trident topologies  Usage: – $ bin/storm sql <sql-file> <topo-name> Storm implementation
  • 49. 49 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL  Example CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders’ CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders’ INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
  • 50. 50 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL  Storm SQL workflow
  • 51. 51 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Streaming SQL  Storm SQL workflow
  • 52. 52 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Storm Usability Improvements Enhanced Debugging and Monitoring of Topologies
  • 53. 53 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Storm usability improvements ./bin/storm set_log_level [topology name]-l [logger name]=[LEVEL]:[TIMEOUT] Dynamic Log Levels Storm CLI: Storm UI:
  • 54. 54 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Storm usability improvements  No more debug bolts or Trident functions  Click on the “Events” link to view the sample log. Tuple Sampling
  • 55. 55 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Storm usability improvements  Distributed Log search – Search across all topology logs, even archived (ZIP) logs.  Dynamic worker profiling – Heap dumps, JStack output, Profiler recordings  Restart workers from UI  Supervisor health checks
  • 56. 56 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Integrations New  Cassandra  Solr  Elastic Search  MQTT Improvements  Kafka  HDFS spout  Hbase  Hive
  • 57. 57 © Hortonworks Inc. 2011 – 2016. All Rights Reserved What’s next for Storm? 2.0 and Beyond
  • 58. 58 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Clojure to Java Broadening the contributor base Alibaba JStorm Contribution
  • 59. 59 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Apache Beam (incubating) Integration  Unified API for dealing with bounded/unbounded data sources (i.e. batch/streaming)  One API. Multiple implementations (execution engines). Called “Runners” in Beamspeak.
  • 60. 60 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Apache Storm 2.0 and beyond  Windowing enhancements  Unified Stream API  Revamped UI  Metrics improvements and many more…
  • 61. 61 © Hortonworks Inc. 2011 – 2016. All Rights Reserved Q&A Thank You

Editor's Notes

  1. Anchoring / Acking is automatically handled
  2. This is currently work in progress. The beta version was shipped with 1.0 and is being currently enhanced.