SlideShare une entreprise Scribd logo
1  sur  47
Télécharger pour lire hors ligne
Real-time Streams & Logs
Andrew Montalenti, CTO
Keith Bourgoin, Backend Lead
1 of 47
Agenda
Parse.ly problem space
Aggregating the stream (Storm)
Organizing around logs (Kafka)
2 of 47
Admin
Our presentations and code:
http://parse.ly/code
This presentation's slides:
http://parse.ly/slides/logs
This presentation's notes:
http://parse.ly/slides/logs/notes
3 of 47
What is Parse.ly?
4 of 47
What is Parse.ly?
Web content analytics for digital storytellers.
5 of 47
Velocity
Average post has <48-hour shelf life.
6 of 47
Volume
Top publishers write 1000's of posts per day.
7 of 47
Time series data
8 of 47
Summary data
9 of 47
Ranked data
10 of 47
Benchmark data
11 of 47
Information radiators
12 of 47
Architecture evolution
13 of 47
Queues and workers
Queues: RabbitMQ => Redis => ZeroMQ
Workers: Cron Jobs => Celery
14 of 47
Workers and databases
15 of 47
Lots of moving parts
16 of 47
In short: it started to get messy
17 of 47
Introducing Storm
Storm is a distributed real-time computation system.
Hadoop provides a set of general primitives for doing batch
processing.
Storm provides a set of general primitives for doing
real-time computation.
Perfect as a replacement for ad-hoc workers-and-queues
systems.
18 of 47
Storm features
Speed
Fault tolerance
Parallelism
Guaranteed Messages
Easy Code Management
Local Dev
19 of 47
Storm primitives
Streaming Data Set, typically from Kafka.
ZeroMQ used for inter-process communication.
Bolts & Spouts; Storm's Topology is a DAG.
Nimbus & Workers manage execution.
Tuneable parallelism + built-in fault tolerance.
20 of 47
Wired Topology
21 of 47
Tuple Tree
Tuple tree, anchoring, and retries.
22 of 47
Word Stream Spout (Storm)
;; spout configuration
{"word-spout" (shell-spout-spec
;; Python Spout implementation:
;; - fetches words (e.g. from Kafka)
["python" "words.py"]
;; - emits (word,) tuples
["word"]
)
}
23 of 47
Word Stream Spout in Python
import itertools
from streamparse import storm
class WordSpout(storm.Spout):
def initialize(self, conf, ctx):
self.words = itertools.cycle(['dog', 'cat',
'zebra', 'elephant'])
def next_tuple(self):
word = next(self.words)
storm.emit([word])
WordSpout().run()
24 of 47
Word Count Bolt (Storm)
;; bolt configuration
{"count-bolt" (shell-bolt-spec
;; Bolt input: Spout and field grouping on word
{"word-spout" ["word"]}
;; Python Bolt implementation:
;; - maintains a Counter of word
;; - increments as new words arrive
["python" "wordcount.py"]
;; Emits latest word count for most recent word
["word" "count"]
;; parallelism = 2
:p 2
)
}
25 of 47
Word Count Bolt in Python
from collections import Counter
from streamparse import storm
class WordCounter(storm.Bolt):
def initialize(self, conf, ctx):
self.counts = Counter()
def process(self, tup):
word = tup.values[0]
self.counts[word] += 1
storm.emit([word, self.counts[word]])
storm.log('%s: %d' % (word, self.counts[word]))
WordCounter().run()
26 of 47
streamparse
sparse provides a CLI front-end to streamparse, a
framework for creating Python projects for running,
debugging, and submitting Storm topologies for data
processing. (still in development)
After installing the lein (only dependency), you can run:
pip install streamparse
This will offer a command-line tool, sparse. Use:
sparse quickstart
27 of 47
Running and debugging
You can then run the local Storm topology using:
$ sparse run
Running wordcount topology...
Options: {:spec "topologies/wordcount.clj", ...}
#<StormTopology StormTopology(spouts:{word-spout=...
storm.daemon.nimbus - Starting Nimbus with conf {...
storm.daemon.supervisor - Starting supervisor with id 4960ac74...
storm.daemon.nimbus - Received topology submission with conf {...
... lots of output as topology runs...
Interested? Lightning talk!
28 of 47
Organizing around logs
29 of 47
Not all logs are application logs
A "log" could be any stream of structured data:
Web logs
Raw data waiting to be processed
Partially processed data
Database operations (e.g. mongo's oplog)
A series of timestamped facts about a given system.
30 of 47
LinkedIn's lattice problem
31 of 47
Enter the unified log
32 of 47
Log-centric is simpler
33 of 47
Parse.ly is log-centric, too
34 of 47
Introducing Apache Kafka
Log-centric messaging system developed at LinkedIn.
Designed for throughput; efficient resource use.
Persists to disk; in-memory for recent data
Little to no overhead for new consumers
Scalable to 10,000's of messages per second
As of 0.8, full replication of topic data.
35 of 47
Kafka concepts
Concept Description
Cluster An arrangement of Brokers & Zookeeper
nodes
Broker An individual node in the Cluster
Topic A group of related messages (a stream)
Partition Part of a topic, used for replication
Producer Publishes messages to stream
Consumer
Group
Group of related processes reading a topic
Offset Point in a topic that the consumer has read to
36 of 47
What's the catch?
Replication isn't perfect. Network partitions can cause
problems.
No out-of-order acknowledgement:
"Offset" is a marker of where consumer is in log;
nothing more.
On a restart, you know where to start reading, but
not if individual messages before the stored offset
was fully processed.
In practice, not as much of a problem as it sounds.
37 of 47
Kafka is a "distributed log"
Topics are logs, not queues.
Consumers read into offsets of the log.
Logs are maintained for a configurable period of time.
Messages can be "replayed".
Consumers can share identical logs easily.
38 of 47
Multi-consumer
Even if Kafka's availability and scalability story isn't
interesting to you, the multi-consumer story should be.
39 of 47
Queue problems, revisited
Traditional queues (e.g. RabbitMQ / Redis):
not distributed / highly available at core
not persistent ("overflows" easily)
more consumers mean more queue server load
Kafka solves all of these problems.
40 of 47
Kafka + Storm
Good fit for at-least-once processing.
No need for out-of-order acks.
Community work is ongoing for at-most-once processing.
Able to keep up with Storm's high-throughput processing.
Great for handling backpressure during traffic spikes.
41 of 47
Kafka in Python (1)
python-kafka (0.8+)
https://github.com/mumrah/kafka-python
from kafka.client import KafkaClient
from kafka.consumer import SimpleConsumer
kafka = KafkaClient('localhost:9092')
consumer = SimpleConsumer(kafka, 'test_consumer', 'raw_data')
start = time.time()
for msg in consumer:
count += 1
if count % 1000 == 0:
dur = time.time() - start
print 'Reading at {:.2f} messages/sec'.format(dur/1000)
start = time.time()
42 of 47
Kafka in Python (2)
samsa (0.7x)
https://github.com/getsamsa/samsa
import time
from kazoo.client import KazooClient
from samsa.cluster import Cluster
zk = KazooClient()
zk.start()
cluster = Cluster(zk)
queue = cluster.topics['raw_data'].subscribe('test_consumer')
start = time.time()
for msg in queue:
count += 1
if count % 1000 == 0:
dur = time.time() - start
print 'Reading at {:.2f} messages/sec'.format(dur/1000)
queue.commit_offsets() # commit to zk every 1k msgs
43 of 47
Other Log-Centric Companies
Company Logs Workers
LinkedIn Kafka* Samza
Twitter Kafka Storm*
Pinterest Kafka Storm
Spotify Kafka Storm
Wikipedia Kafka Storm
Outbrain Kafka Storm
LivePerson Kafka Storm
Netflix Kafka ???
44 of 47
Conclusion
45 of 47
What we've learned
There is no silver bullet data processing technology.
Log storage is very cheap, and getting cheaper.
"Timestamped facts" is rawest form of data available.
Storm and Kafka allow you to develop atop those facts.
Organizing around real-time logs is a wise decision.
46 of 47
Questions?
Go forth and stream!
Parse.ly:
http://parse.ly/code
http://twitter.com/parsely
Andrew & Keith:
http://twitter.com/amontalenti
http://twitter.com/kbourgoin
47 of 47

Contenu connexe

Tendances

Real-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormReal-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormDavorin Vukelic
 
Realtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQRealtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQXin Wang
 
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
 
Multi-Tenant Storm Service on Hadoop Grid
Multi-Tenant Storm Service on Hadoop GridMulti-Tenant Storm Service on Hadoop Grid
Multi-Tenant Storm Service on Hadoop GridDataWorks Summit
 
Real-Time Big Data at In-Memory Speed, Using Storm
Real-Time Big Data at In-Memory Speed, Using StormReal-Time Big Data at In-Memory Speed, Using Storm
Real-Time Big Data at In-Memory Speed, Using StormNati Shalom
 
Developing Java Streaming Applications with Apache Storm
Developing Java Streaming Applications with Apache StormDeveloping Java Streaming Applications with Apache Storm
Developing Java Streaming Applications with Apache StormLester Martin
 
Storm: distributed and fault-tolerant realtime computation
Storm: distributed and fault-tolerant realtime computationStorm: distributed and fault-tolerant realtime computation
Storm: distributed and fault-tolerant realtime computationnathanmarz
 
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 - VerisignMichael Noll
 
Storm presentation
Storm presentationStorm presentation
Storm presentationShyam Raj
 
Realtime Analytics with Storm and Hadoop
Realtime Analytics with Storm and HadoopRealtime Analytics with Storm and Hadoop
Realtime Analytics with Storm and HadoopDataWorks Summit
 
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 ArchitectureP. Taylor Goetz
 
Slide #1:Introduction to Apache Storm
Slide #1:Introduction to Apache StormSlide #1:Introduction to Apache Storm
Slide #1:Introduction to Apache StormMd. Shamsur Rahim
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache StormP. Taylor Goetz
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and howPetr Zapletal
 
Storm: The Real-Time Layer - GlueCon 2012
Storm: The Real-Time Layer  - GlueCon 2012Storm: The Real-Time Layer  - GlueCon 2012
Storm: The Real-Time Layer - GlueCon 2012Dan Lynn
 

Tendances (20)

Real-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormReal-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache Storm
 
Realtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQRealtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQ
 
Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)Scaling Apache Storm (Hadoop Summit 2015)
Scaling Apache Storm (Hadoop Summit 2015)
 
Introduction to Storm
Introduction to StormIntroduction to Storm
Introduction to Storm
 
Multi-Tenant Storm Service on Hadoop Grid
Multi-Tenant Storm Service on Hadoop GridMulti-Tenant Storm Service on Hadoop Grid
Multi-Tenant Storm Service on Hadoop Grid
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 
Apache Storm Internals
Apache Storm InternalsApache Storm Internals
Apache Storm Internals
 
Real-Time Big Data at In-Memory Speed, Using Storm
Real-Time Big Data at In-Memory Speed, Using StormReal-Time Big Data at In-Memory Speed, Using Storm
Real-Time Big Data at In-Memory Speed, Using Storm
 
Developing Java Streaming Applications with Apache Storm
Developing Java Streaming Applications with Apache StormDeveloping Java Streaming Applications with Apache Storm
Developing Java Streaming Applications with Apache Storm
 
Storm: distributed and fault-tolerant realtime computation
Storm: distributed and fault-tolerant realtime computationStorm: distributed and fault-tolerant realtime computation
Storm: distributed and fault-tolerant realtime computation
 
Storm
StormStorm
Storm
 
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
 
Storm presentation
Storm presentationStorm presentation
Storm presentation
 
Realtime Analytics with Storm and Hadoop
Realtime Analytics with Storm and HadoopRealtime Analytics with Storm and Hadoop
Realtime Analytics with Storm and Hadoop
 
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
 
Storm and Cassandra
Storm and Cassandra Storm and Cassandra
Storm and Cassandra
 
Slide #1:Introduction to Apache Storm
Slide #1:Introduction to Apache StormSlide #1:Introduction to Apache Storm
Slide #1:Introduction to Apache Storm
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and how
 
Storm: The Real-Time Layer - GlueCon 2012
Storm: The Real-Time Layer  - GlueCon 2012Storm: The Real-Time Layer  - GlueCon 2012
Storm: The Real-Time Layer - GlueCon 2012
 

En vedette

Real time analytics with Kafka and SparkStreaming
Real time analytics with Kafka and SparkStreamingReal time analytics with Kafka and SparkStreaming
Real time analytics with Kafka and SparkStreamingAshish Singh
 
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 realtimeGuido Schmutz
 
streamparse and pystorm: simple reliable parallel processing with storm
streamparse and pystorm: simple reliable parallel processing with stormstreamparse and pystorm: simple reliable parallel processing with storm
streamparse and pystorm: simple reliable parallel processing with stormDaniel Blanchard
 
Spark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingSpark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingJack Gudenkauf
 
How Apache Kafka is transforming Hadoop, Spark and Storm
How Apache Kafka is transforming Hadoop, Spark and StormHow Apache Kafka is transforming Hadoop, Spark and Storm
How Apache Kafka is transforming Hadoop, Spark and StormEdureka!
 
Volta: Logging, Metrics, and Monitoring as a Service
Volta: Logging, Metrics, and Monitoring as a ServiceVolta: Logging, Metrics, and Monitoring as a Service
Volta: Logging, Metrics, and Monitoring as a ServiceLN Renganarayana
 
Data Streaming with Apache Kafka & MongoDB
Data Streaming with Apache Kafka & MongoDBData Streaming with Apache Kafka & MongoDB
Data Streaming with Apache Kafka & MongoDBconfluent
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingRobert Metzger
 
Hadoop Reporting and Analysis - Jaspersoft
Hadoop Reporting and Analysis - JaspersoftHadoop Reporting and Analysis - Jaspersoft
Hadoop Reporting and Analysis - JaspersoftHortonworks
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark StreamingP. Taylor Goetz
 

En vedette (12)

Real time analytics with Kafka and SparkStreaming
Real time analytics with Kafka and SparkStreamingReal time analytics with Kafka and SparkStreaming
Real time analytics with Kafka and SparkStreaming
 
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
 
streamparse and pystorm: simple reliable parallel processing with storm
streamparse and pystorm: simple reliable parallel processing with stormstreamparse and pystorm: simple reliable parallel processing with storm
streamparse and pystorm: simple reliable parallel processing with storm
 
Spark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingSpark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream Processing
 
How Apache Kafka is transforming Hadoop, Spark and Storm
How Apache Kafka is transforming Hadoop, Spark and StormHow Apache Kafka is transforming Hadoop, Spark and Storm
How Apache Kafka is transforming Hadoop, Spark and Storm
 
Volta: Logging, Metrics, and Monitoring as a Service
Volta: Logging, Metrics, and Monitoring as a ServiceVolta: Logging, Metrics, and Monitoring as a Service
Volta: Logging, Metrics, and Monitoring as a Service
 
Data Streaming with Apache Kafka & MongoDB
Data Streaming with Apache Kafka & MongoDBData Streaming with Apache Kafka & MongoDB
Data Streaming with Apache Kafka & MongoDB
 
Apache Metron: Community Driven Cyber Security
Apache Metron: Community Driven Cyber Security Apache Metron: Community Driven Cyber Security
Apache Metron: Community Driven Cyber Security
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer Checkpointing
 
Hadoop Reporting and Analysis - Jaspersoft
Hadoop Reporting and Analysis - JaspersoftHadoop Reporting and Analysis - Jaspersoft
Hadoop Reporting and Analysis - Jaspersoft
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark Streaming
 
Cisco OpenSOC
Cisco OpenSOCCisco OpenSOC
Cisco OpenSOC
 

Similaire à Real-time streams and logs with Storm and Kafka

Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustEvan Chan
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka StreamsGuozhang Wang
 
Apache Kafka, and the Rise of Stream Processing
Apache Kafka, and the Rise of Stream ProcessingApache Kafka, and the Rise of Stream Processing
Apache Kafka, and the Rise of Stream ProcessingGuozhang Wang
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...Data Con LA
 
Project Reactor Now and Tomorrow
Project Reactor Now and TomorrowProject Reactor Now and Tomorrow
Project Reactor Now and TomorrowVMware Tanzu
 
Princeton Dec 2022 Meetup_ StreamNative and Cloudera Streaming
Princeton Dec 2022 Meetup_ StreamNative and Cloudera StreamingPrinceton Dec 2022 Meetup_ StreamNative and Cloudera Streaming
Princeton Dec 2022 Meetup_ StreamNative and Cloudera StreamingTimothy Spann
 
Splunk Conf 2014 - Getting the message
Splunk Conf 2014 - Getting the messageSplunk Conf 2014 - Getting the message
Splunk Conf 2014 - Getting the messageDamien Dallimore
 
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarTimothy Spann
 
SC'18 BoF Presentation
SC'18 BoF PresentationSC'18 BoF Presentation
SC'18 BoF Presentationrcastain
 
Sedna XML Database: Executor Internals
Sedna XML Database: Executor InternalsSedna XML Database: Executor Internals
Sedna XML Database: Executor InternalsIvan Shcheklein
 
LibOS as a regression test framework for Linux networking #netdev1.1
LibOS as a regression test framework for Linux networking #netdev1.1LibOS as a regression test framework for Linux networking #netdev1.1
LibOS as a regression test framework for Linux networking #netdev1.1Hajime Tazaki
 
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE confluent
 
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINEKafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINEkawamuray
 
Building Modern Data Streaming Apps with Python
Building Modern Data Streaming Apps with PythonBuilding Modern Data Streaming Apps with Python
Building Modern Data Streaming Apps with PythonTimothy Spann
 
Serverless Event Streaming Applications as Functionson K8
Serverless Event Streaming Applications as Functionson K8Serverless Event Streaming Applications as Functionson K8
Serverless Event Streaming Applications as Functionson K8Timothy Spann
 
Developing Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache KafkaDeveloping Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache KafkaJoe Stein
 
I can't believe it's not a queue: Kafka and Spring
I can't believe it's not a queue: Kafka and SpringI can't believe it's not a queue: Kafka and Spring
I can't believe it's not a queue: Kafka and SpringJoe Kutner
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonTimothy Spann
 

Similaire à Real-time streams and logs with Storm and Kafka (20)

Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to Rust
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
Apache Kafka, and the Rise of Stream Processing
Apache Kafka, and the Rise of Stream ProcessingApache Kafka, and the Rise of Stream Processing
Apache Kafka, and the Rise of Stream Processing
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...
Data Con LA 2018 - A Serverless Approach to Data Processing using Apache Puls...
 
Project Reactor Now and Tomorrow
Project Reactor Now and TomorrowProject Reactor Now and Tomorrow
Project Reactor Now and Tomorrow
 
Princeton Dec 2022 Meetup_ StreamNative and Cloudera Streaming
Princeton Dec 2022 Meetup_ StreamNative and Cloudera StreamingPrinceton Dec 2022 Meetup_ StreamNative and Cloudera Streaming
Princeton Dec 2022 Meetup_ StreamNative and Cloudera Streaming
 
Splunk Conf 2014 - Getting the message
Splunk Conf 2014 - Getting the messageSplunk Conf 2014 - Getting the message
Splunk Conf 2014 - Getting the message
 
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
 
SC'18 BoF Presentation
SC'18 BoF PresentationSC'18 BoF Presentation
SC'18 BoF Presentation
 
Sedna XML Database: Executor Internals
Sedna XML Database: Executor InternalsSedna XML Database: Executor Internals
Sedna XML Database: Executor Internals
 
LibOS as a regression test framework for Linux networking #netdev1.1
LibOS as a regression test framework for Linux networking #netdev1.1LibOS as a regression test framework for Linux networking #netdev1.1
LibOS as a regression test framework for Linux networking #netdev1.1
 
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy—160 Billion Daily Messages on One Shared Cluster at LINE
 
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINEKafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINE
Kafka Multi-Tenancy - 160 Billion Daily Messages on One Shared Cluster at LINE
 
Building Modern Data Streaming Apps with Python
Building Modern Data Streaming Apps with PythonBuilding Modern Data Streaming Apps with Python
Building Modern Data Streaming Apps with Python
 
Typesafe spark- Zalando meetup
Typesafe spark- Zalando meetupTypesafe spark- Zalando meetup
Typesafe spark- Zalando meetup
 
Serverless Event Streaming Applications as Functionson K8
Serverless Event Streaming Applications as Functionson K8Serverless Event Streaming Applications as Functionson K8
Serverless Event Streaming Applications as Functionson K8
 
Developing Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache KafkaDeveloping Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache Kafka
 
I can't believe it's not a queue: Kafka and Spring
I can't believe it's not a queue: Kafka and SpringI can't believe it's not a queue: Kafka and Spring
I can't believe it's not a queue: Kafka and Spring
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with Python
 

Dernier

New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Farhan Tariq
 
A Framework for Development in the AI Age
A Framework for Development in the AI AgeA Framework for Development in the AI Age
A Framework for Development in the AI AgeCprime
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxLoriGlavin3
 
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Alkin Tezuysal
 
Generative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersGenerative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersRaghuram Pandurangan
 
Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Hiroshi SHIBATA
 
Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rick Flair
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteDianaGray10
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfIngrid Airi González
 
Decarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityDecarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityIES VE
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity PlanDatabarracks
 
[Webinar] SpiraTest - Setting New Standards in Quality Assurance
[Webinar] SpiraTest - Setting New Standards in Quality Assurance[Webinar] SpiraTest - Setting New Standards in Quality Assurance
[Webinar] SpiraTest - Setting New Standards in Quality AssuranceInflectra
 
Manual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditManual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditSkynet Technologies
 
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentPim van der Noll
 
A Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersA Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersNicole Novielli
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxLoriGlavin3
 
2024 April Patch Tuesday
2024 April Patch Tuesday2024 April Patch Tuesday
2024 April Patch TuesdayIvanti
 
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfpanagenda
 

Dernier (20)

New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...
 
A Framework for Development in the AI Age
A Framework for Development in the AI AgeA Framework for Development in the AI Age
A Framework for Development in the AI Age
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
 
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
 
Generative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersGenerative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information Developers
 
Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024
 
Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test Suite
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdf
 
Decarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityDecarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a reality
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity Plan
 
[Webinar] SpiraTest - Setting New Standards in Quality Assurance
[Webinar] SpiraTest - Setting New Standards in Quality Assurance[Webinar] SpiraTest - Setting New Standards in Quality Assurance
[Webinar] SpiraTest - Setting New Standards in Quality Assurance
 
Manual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditManual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance Audit
 
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
 
A Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersA Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software Developers
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
 
2024 April Patch Tuesday
2024 April Patch Tuesday2024 April Patch Tuesday
2024 April Patch Tuesday
 
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
 

Real-time streams and logs with Storm and Kafka

  • 1. Real-time Streams & Logs Andrew Montalenti, CTO Keith Bourgoin, Backend Lead 1 of 47
  • 2. Agenda Parse.ly problem space Aggregating the stream (Storm) Organizing around logs (Kafka) 2 of 47
  • 3. Admin Our presentations and code: http://parse.ly/code This presentation's slides: http://parse.ly/slides/logs This presentation's notes: http://parse.ly/slides/logs/notes 3 of 47
  • 5. What is Parse.ly? Web content analytics for digital storytellers. 5 of 47
  • 6. Velocity Average post has <48-hour shelf life. 6 of 47
  • 7. Volume Top publishers write 1000's of posts per day. 7 of 47
  • 14. Queues and workers Queues: RabbitMQ => Redis => ZeroMQ Workers: Cron Jobs => Celery 14 of 47
  • 16. Lots of moving parts 16 of 47
  • 17. In short: it started to get messy 17 of 47
  • 18. Introducing Storm Storm is a distributed real-time computation system. Hadoop provides a set of general primitives for doing batch processing. Storm provides a set of general primitives for doing real-time computation. Perfect as a replacement for ad-hoc workers-and-queues systems. 18 of 47
  • 19. Storm features Speed Fault tolerance Parallelism Guaranteed Messages Easy Code Management Local Dev 19 of 47
  • 20. Storm primitives Streaming Data Set, typically from Kafka. ZeroMQ used for inter-process communication. Bolts & Spouts; Storm's Topology is a DAG. Nimbus & Workers manage execution. Tuneable parallelism + built-in fault tolerance. 20 of 47
  • 22. Tuple Tree Tuple tree, anchoring, and retries. 22 of 47
  • 23. Word Stream Spout (Storm) ;; spout configuration {"word-spout" (shell-spout-spec ;; Python Spout implementation: ;; - fetches words (e.g. from Kafka) ["python" "words.py"] ;; - emits (word,) tuples ["word"] ) } 23 of 47
  • 24. Word Stream Spout in Python import itertools from streamparse import storm class WordSpout(storm.Spout): def initialize(self, conf, ctx): self.words = itertools.cycle(['dog', 'cat', 'zebra', 'elephant']) def next_tuple(self): word = next(self.words) storm.emit([word]) WordSpout().run() 24 of 47
  • 25. Word Count Bolt (Storm) ;; bolt configuration {"count-bolt" (shell-bolt-spec ;; Bolt input: Spout and field grouping on word {"word-spout" ["word"]} ;; Python Bolt implementation: ;; - maintains a Counter of word ;; - increments as new words arrive ["python" "wordcount.py"] ;; Emits latest word count for most recent word ["word" "count"] ;; parallelism = 2 :p 2 ) } 25 of 47
  • 26. Word Count Bolt in Python from collections import Counter from streamparse import storm class WordCounter(storm.Bolt): def initialize(self, conf, ctx): self.counts = Counter() def process(self, tup): word = tup.values[0] self.counts[word] += 1 storm.emit([word, self.counts[word]]) storm.log('%s: %d' % (word, self.counts[word])) WordCounter().run() 26 of 47
  • 27. streamparse sparse provides a CLI front-end to streamparse, a framework for creating Python projects for running, debugging, and submitting Storm topologies for data processing. (still in development) After installing the lein (only dependency), you can run: pip install streamparse This will offer a command-line tool, sparse. Use: sparse quickstart 27 of 47
  • 28. Running and debugging You can then run the local Storm topology using: $ sparse run Running wordcount topology... Options: {:spec "topologies/wordcount.clj", ...} #<StormTopology StormTopology(spouts:{word-spout=... storm.daemon.nimbus - Starting Nimbus with conf {... storm.daemon.supervisor - Starting supervisor with id 4960ac74... storm.daemon.nimbus - Received topology submission with conf {... ... lots of output as topology runs... Interested? Lightning talk! 28 of 47
  • 30. Not all logs are application logs A "log" could be any stream of structured data: Web logs Raw data waiting to be processed Partially processed data Database operations (e.g. mongo's oplog) A series of timestamped facts about a given system. 30 of 47
  • 32. Enter the unified log 32 of 47
  • 34. Parse.ly is log-centric, too 34 of 47
  • 35. Introducing Apache Kafka Log-centric messaging system developed at LinkedIn. Designed for throughput; efficient resource use. Persists to disk; in-memory for recent data Little to no overhead for new consumers Scalable to 10,000's of messages per second As of 0.8, full replication of topic data. 35 of 47
  • 36. Kafka concepts Concept Description Cluster An arrangement of Brokers & Zookeeper nodes Broker An individual node in the Cluster Topic A group of related messages (a stream) Partition Part of a topic, used for replication Producer Publishes messages to stream Consumer Group Group of related processes reading a topic Offset Point in a topic that the consumer has read to 36 of 47
  • 37. What's the catch? Replication isn't perfect. Network partitions can cause problems. No out-of-order acknowledgement: "Offset" is a marker of where consumer is in log; nothing more. On a restart, you know where to start reading, but not if individual messages before the stored offset was fully processed. In practice, not as much of a problem as it sounds. 37 of 47
  • 38. Kafka is a "distributed log" Topics are logs, not queues. Consumers read into offsets of the log. Logs are maintained for a configurable period of time. Messages can be "replayed". Consumers can share identical logs easily. 38 of 47
  • 39. Multi-consumer Even if Kafka's availability and scalability story isn't interesting to you, the multi-consumer story should be. 39 of 47
  • 40. Queue problems, revisited Traditional queues (e.g. RabbitMQ / Redis): not distributed / highly available at core not persistent ("overflows" easily) more consumers mean more queue server load Kafka solves all of these problems. 40 of 47
  • 41. Kafka + Storm Good fit for at-least-once processing. No need for out-of-order acks. Community work is ongoing for at-most-once processing. Able to keep up with Storm's high-throughput processing. Great for handling backpressure during traffic spikes. 41 of 47
  • 42. Kafka in Python (1) python-kafka (0.8+) https://github.com/mumrah/kafka-python from kafka.client import KafkaClient from kafka.consumer import SimpleConsumer kafka = KafkaClient('localhost:9092') consumer = SimpleConsumer(kafka, 'test_consumer', 'raw_data') start = time.time() for msg in consumer: count += 1 if count % 1000 == 0: dur = time.time() - start print 'Reading at {:.2f} messages/sec'.format(dur/1000) start = time.time() 42 of 47
  • 43. Kafka in Python (2) samsa (0.7x) https://github.com/getsamsa/samsa import time from kazoo.client import KazooClient from samsa.cluster import Cluster zk = KazooClient() zk.start() cluster = Cluster(zk) queue = cluster.topics['raw_data'].subscribe('test_consumer') start = time.time() for msg in queue: count += 1 if count % 1000 == 0: dur = time.time() - start print 'Reading at {:.2f} messages/sec'.format(dur/1000) queue.commit_offsets() # commit to zk every 1k msgs 43 of 47
  • 44. Other Log-Centric Companies Company Logs Workers LinkedIn Kafka* Samza Twitter Kafka Storm* Pinterest Kafka Storm Spotify Kafka Storm Wikipedia Kafka Storm Outbrain Kafka Storm LivePerson Kafka Storm Netflix Kafka ??? 44 of 47
  • 46. What we've learned There is no silver bullet data processing technology. Log storage is very cheap, and getting cheaper. "Timestamped facts" is rawest form of data available. Storm and Kafka allow you to develop atop those facts. Organizing around real-time logs is a wise decision. 46 of 47
  • 47. Questions? Go forth and stream! Parse.ly: http://parse.ly/code http://twitter.com/parsely Andrew & Keith: http://twitter.com/amontalenti http://twitter.com/kbourgoin 47 of 47