SlideShare ist ein Scribd-Unternehmen logo
1 von 55
Pravega
Storage Reimagined for a Streaming World
Srikanth Satya & Tom Kaitchuck
Dell EMC Unstructured Storage
srikanth.satya@emc.com
tom.kaitchuck@emc.com
Streaming is Disruptive
How do you shrink to zero the time it takes to turn
massive volumes of data into information and action?
No problem! Just process data as it arrives, quickly, and
in a continuous and infinite fashion
Demands disruptive systems capabilities
▪ Ability to treat data as continuous and infinite
▪ Ability to dynamically scale ingest, storage, and processing in
coordination with data arrival volume
▪ Ability to deliver accurate results processing continuously
even with late arriving or out of order data
Stateful processors born for streaming, like
Apache Flink, are disrupting how we think
about data computing …
We think the world needs a complementary
technology … to similarly disrupt storage.
Introducing Pravega Streams
A new storage abstraction – a stream – for continuous and infinite data
▪ Named, durable, append-only, infinite sequence of bytes
▪ With low-latency appends to and reads from the tail of the sequence
▪ With high-throughput reads for older portions of the sequence
Coordinated scaling of stream storage and stream processing
▪ Stream writes partitioned by app key
▪ Stream reads independently and automatically partitioned by arrival rate SLO
▪ Scaling protocol to allow stream processors to scale in lockstep with storage
Enabling system-wide exactly once processing across multiple apps
▪ Streams are ordered and strongly consistent
▪ Chain independent streaming apps via streams
▪ Stream transactions integrate with checkpoint schemes such as the one used in Flink
In Place of All This …
Stateless
Real-Time Processing
Batch
Processing
Messaging
Storage
Analytics
Storage
Fast, Approximate
Slow, Accurate
Archive
Storage
New Data
Old Data
… Just Do This!
Streaming
Storage
Fast, Accurate
Stateful
Stream Processing
New & Old Data
Pravega Flink
Each component in the combined system – writers, streams, readers,
apps – is independently, elastically, and dynamically scalable in
coordination with data volume arrival rate over time. Sweet!
Pravega Streams + Flink
First
Flink App
Worker
WorkerRaw
Stream …
Worker
…
Segment
Protocol coordination between
streaming storage and stream
processor to dynamically scale up
and down the number of segments
and Flink workers based on load
variance over time
Flink utilizes a stream’s transactional writes to extend exactly
once processing across multiple, chained apps
Social,IoT,…
Writers
Cooked
Stream
Next
Flink App
Sink
Segment
Segment
Sink …
Writers scale based on app configuration; stream
storage elastically and independently scales based
on aggregate incoming volume of data
WorkerSegment
WorkerSegment
…
And It’s Just the Beginning …
Ingest Buffer
& Pub/Sub
Persistent
Data Structures
Search
Streaming
Analytics
Cloud-Scale Storage
Pravega Streams
Enabling a new generation of distributed middleware
reimagined as streaming infrastructure
How Pravega Works
Architecture & System Design
Pravega Architecture Goals
• All data is durable
– Data is replicated and persisted to disk before being acknowledged
• Strict ordering guarantees and exactly once semantics
– Across both tail and catch-up reads
– Client tracks read offset, Producers use transactions
• Lightweight, elastic, infinite, high performance
– Support tens of millions of streams
– Low (<10ms) latency writes; throughput bounded by network bandwidth
– Read pattern (e.g. many catch-up reads) doesn’t affect write performance
• Dynamic partitioning of streams based on load and throughput SLO
• Capacity is not bounded by the size of a single node
Streaming model
• Fundamental data structure is an
ordered sequence of bytes
• Think of it as a durable socket or Unix
pipe
• Bytes are not interpreted server side
• This implicitly guarantees order and
non-duplication
• Higher layers impose further structure,
e.g. message boundaries
M5 M4 M3 M2 M1
Pravega
Stream
Cartoon API
public interface SegmentWriter {
/** Asynchronously and atomically write data
*/
void write(ByteBuffer data);
/** Asynchronously and atomically write the
data if it can be written at the provided offset
*/
void write(ByteBuffer data, long atOffset);
/** Asynchronously and atomically write all of
the data from the provided input stream */
void write(InputStream in);
}
public interface SegmentReader {
long fetchCurrentLength();
/** Returns the current offset */
long getOffset();
/** Sets the next offset to read from */
void setOffset(long offset);
/** Read bytes from the current offset */
ByteBuffer read(int length);
}
Idempotent Append
Writer
Append {0101011} and
Assign appendNumber = 7
Pravega
Stream
Idempotent Append
Writer
What is appendNumber?
7
Pravega
Stream
Idempotent Append
Writer
Append {0101011} and Assign:
appendNumber = 7
If and only if appendNumber < 7
Pravega
Stream
Pravega
Stream
Idempotent output
source Sinksink
Pravega
Stream
Sinksink
Idempotent output
source
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Architecture overview - Write
Client
write(data1)
write(data2)
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Commodity Server
Bookkeeper
Pravega
Cache
SSD
SSD
HDFS
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Architecture overview - Read
Client
read()
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Commodity Server
Bookkeeper
Pravega
Cache
SSD
SSD
HDFS
HDFS
Commodity Server
Pravega
Bookkeeper
Architecture overview - Evict
Commodity Server
Pravega
Bookkeeper
Commodity Server
Bookkeeper
Pravega
• Files in HDFS are organized by Stream Segment
• Read-ahead cache optimizations are employed
HDFS
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Architecture overview - Read
Client
read()
Commodity Server
Pravega
Bookkeeper
Cache
SSD
SSD
Commodity Server
Bookkeeper
Pravega
Cache
SSD
SSD
Commodity Server
Pravega
Bookkeeper
Architecture overview - Recover
Commodity Server
Pravega
Bookkeeper
Commodity Server
Bookkeeper
Pravega
• Data is read from Bookkeeper only in the case of node failure
• Used to reconstitute the cache on the remaining hosts
HDFS
Performance Characteristics
• Fast appends to Bookkeeper
– Data is persisted durably to disk 3x replicated consistently <10ms
• Big block writes to HDFS
– Data is mostly cold so it can be erasure encoded and stored cheaply
– If data is read, the job is likely a backfill so we can use a large read-ahead
• A stream’s capacity is not limited by the capacity of a single machine
• Throughput shouldn’t be either …
Scaling: Segment Splitting & Merging
Segment 7
Segment 4
Segment 3
Segment 1
Segment 2
Segment 0
Time
3
2
1
0
t0 t1 t2 t3
Segment 6
Segment 5
t4
Data
Keys
Stream S
Pravega
ProducersProducers
Pravega
Writers
Writer
Configuration
ka .. kf ➔ S
Stream S
Stream
Segments
ss0
ss1
ssn
…
ProducersProducers
Pravega
Readers
Scaling: Write Parallelism
Number of stream segments
dynamically changes based
on load and SLO1
Segments are split and
merged dynamically without
manual intervention
2
Writer configurations do not change when
segments are split or merged
3
EventWriter API
/** A writer can write events to a stream. */
public interface EventStreamWriter {
/** Send an event to the stream. Event must appear in the stream exactly once */
AckFuture writeEvent(String routingKey, Type event);
/** Start a new transaction on this stream */
Transaction<Type> beginTxn(long transactionTimeout);
}
Pravega
ProducersProducers
Pravega
Writers
Writer
Configuration
ka .. kf ➔ S
Stream S
Stream
Segments
ss0
ss1
ssn
…
ProducersProducers
Pravega
Readers
Scaling: Read Parallelism
Readers are notified when segments are
split or merged enabling reader
parallelism to scale in response to the
stream scaling
EventReader API
public interface EventStreamReader<T> extends AutoCloseable {
/** Read the next event from the stream, blocking for up to timeout */
EventRead<T> readNextEvent(long timeout);
/**
* Close the reader. The segments owned by this reader will automatically be
* redistributed to the other readers in the group.
*/
void close()
}
Conditional Append
Writer
Append {0101011}
If and only if it would be
appended at offset 123 Pravega
Stream
Writer
Synchronizer API
/** A means to synchronize state between many processes */
public interface StateSynchronizer<StateT> {
/** Gets the state object currently held in memory */
StateT getState();
/** Fetch and apply all updates to bring the local state object up to date */
void fetchUpdates();
/** Creates a new update for the latest state object and applies it atomically */
void updateState(Function<StateT, Update<StateT>> updateGenerator);
}
Pravega
Stream
sink
Transactional output
source
sink
Pravega
Stream
Transactional output
source
EventWriter and Transaction API
/** A writer can write events to a stream. */
public interface EventStreamWriter {
/** Send an event to the stream. Event must appear in the stream exactly once */
AckFuture writeEvent(String routingKey, Type event);
/** Start a new transaction on this stream */
Transaction<Type> beginTxn(long transactionTimeout);
}
public interface Transaction<Type> {
void writeEvent(String routingKey, Type event) throws TxnFailedException;
void commit() throws TxnFailedException;
void abort();
}
Transactions
New Item
Stream 1, Segment 1
New Item
Stream 1, Segment 1, TX-230
New Item
…
Transactions
Writer
Append {110…101} to
Segment-1-txn-1
Pravega
Append {010…111} to
Segment-2-txn-1
Pravega
Controller
Create
txn-1
Commit
txn-1
Commit
txn-1
Pravega
Stream
sink
Pravega
Stream
sink
Transactional output
source
Pravega
Stream
sink
Pravega
Stream
sink
Transactional output
source
Pravega
Stream
sink
Pravega
Stream
sink
Transactional output
source
Pravega
Stream
sink
Pravega
Stream
sink
Transactional output
source
Pravega: Streaming Storage for All
▪ Pravega: an open source project with an open community
▪ To be launched @ Dell EMC World this May 10th
▪ Includes infinite byte stream primitive
▪ Plus an Ingest Buffer with Pub/Sub built on top of streams
▪ And Flink integration!
▪ Visit the Dell EMC booth here @ Flink Forward to learn more
▪ Contact us at pravega@emc.com for even more information!
BB-8 Drawing
➢ Stop by the Dell EMC booth and enter to win
➢ Winner will be chosen after the closing Keynote
➢ Must be present to win
Pravega
Email Pravega@emc.com for the latest news and information on Pravega!
Why a new storage system?
Source Sinkconnect Sink
Sink
connect
Sink
connect
Why a new storage system?
Connector Real Time Exactly once Durability Storage
Capacity
Notes
HDFS No Yes Yes Years
Kafka Yes Source only Yes* (Flushed
but not synced)
Days Writes are replicated but may not
persisted to durable media.
(flush.messages=1 bounds this but
is not recommended)
RabbitMQ Yes Source only Yes* (slowly) Days Durability can be added with a
performance hit
Cassandra No Yes* (If updates
are idempotent)
Yes Years App developers need to write
custom logic to handle duplicate
writes.
Sockets Yes No No None
Flink storage needs
Flink Implications for storage
Guarantee Exactly once Exactly once, consistency
Latency Very Low Low latency writes (<10ms)
Throughput High High throughput
Computation model Streaming Streaming model
Overhead of fault
tolerance mechanism
Low Fast recovery
Long retention
Flow control Natural Data can backlog
Capacity not bounded by single host
Separation of application
logic from fault tolerance
Yes Re-reading data provides consistent results
License Apache 2.0 Open Source and linkable
Shared config
public class SharedConfig<K extends Serializable, V extends Serializable> {
public V getProperty(K key);
public V putPropertyIfAbsent(K key, V value);
public boolean removeProperty(K key, V oldValue);
public boolean replaceProperty(K key, V oldValue, V newValue);
}
Smart Workload Distribution
Pravega
Segment
Container
Segment
Container
Segment
Container
ss1ss0 ss2
ss3 ss4
The hot segment is automatically “split,” and the “child”
segments are re-distributed across the cluster relieving the
hot spot while maximizing utilization of the cluster’s available
IOPs capacity
Streaming Storage System
Architecture
Stream
Abstraction
Pravega Streaming Service
Cloud Scale Storage
(HDFS)
• High-Throughput
• High-Scale, Low-Cost
Low-Latency Storage
Apache Bookkeeper
Auto-Tiering
Cache
(Rocks)
Messaging Apps
Real-Time / Batch / Interactive Predictive Analytics
Stream Processors: Spark, Flink, …
Other Apps & Middleware
Pravega Design Innovations
1. Zero-Touch Dynamic Scaling
- Automatically scale read/write
parallelism based on load and SLO
- No service interruptions
- No manual reconfiguration of clients
- No manual reconfiguration of service
resources
2. Smart Workload Distribution
- No need to over-provision servers
for peak load
3. I/O Path Isolation
- For tail writes
- For tail reads
- For catch-up reads
4. Tiering for “Infinite Streams”
5. Transactions For “Exactly Once”
Stream
Processor
App State
App Logic
Worker
Worker
Pravega Optimizations for Stream Processors
Input
Stream
(Pravega)
…
Worker
…
Segment
Memory-Speed
Storage
Dynamically split input stream into
parallel logs: infinite sequence, low-
latency, durable, re-playable with auto-
tiering from hot to cold storage.
1
Coordinate via protocol between
streaming storage and streaming
engine to systematically scale up
and down the number of logs and
source workers based on load
variance over time
2
Support streaming write COMMIT operation to extend
Exactly Once processing semantics across multiple,
chained applications
3
Social,IoT
Producers
Output
Stream
(Pravega)
Stream
Processor
2nd App
SegmentSink
Segment
Segment
Comparing Pravega and Kafka Design Points
Unlike Kafka, Pravega is designed to be a durable and permanent storage system
Quality Pravega Goal Kafka Design Point
Data Durability Replicated and persisted to disk before ACK Replicated but not persisted to disk before ACK
Strict Ordering Consistent ordering on tail and catch-up reads Messages may get reordered
Exactly Once Producers can use transactions for atomicity Messages may get duplicated
Scale Tens of millions of streams per cluster Thousands of topics per cluster
Elastic Dynamic partitioning of streams based on load and SLO Statically configured partitions
Size
Log size is not bounded by the capacity of any single node
Partition size is bounded by capacity of filesystem on
its hosting node
Transparently migrate/retrieve data from Tier 2 storage for
older parts of the log
External ETL required to move data to Tier 2
storage; no access to data via Kafka once moved
Performance
Low (<10ms) latency durable writes; throughput bounded by
network bandwidth
Low-latency achieved only by reducing replication/
reliability parameters
Read pattern (e.g. many catch-up readers) does not affect
write performance
Read patterns adversely affects write performance
due to reliance on OS filesystem cache
✗
✗
✗
✗
✗
✗
✗
✗
✗
Attributes
Connector Streaming Exactly once Durability Storage Capacity
HDFS No Yes Yes Years
Kafka Yes Source only Yes* (Flushed but not synced) Days
Pravega Yes:

Byte
oriented and
event
oriented
Yes.

With either
idempotent
producers, or
transactions
Yes.
Always flushed and synced, with
low latency.
As much as you can fit
in your HDFS cluster.
Apache
BookKeeper
Architecture overview - Write
Metadata

+ Cache
Check Metadata
1
Record data to Log
2
Update 

Metadata / cache3ACK
4
Architecture overview - Read
Metadata

+ Cache
Apache
BookKeeper
Pull from cache
2.a
Check Metadata
1
Pull from HDFS
2.b
Architecture overview - Evict
Metadata

+ Cache
Apache
BookKeeper
Write contents from cache to HDFS
1
Mark data for removal
2
Architecture overview - Recover
Metadata

+ Cache
Apache
BookKeeper
Take ownership of BK Ledger
1
Re-populate metadata/cache
from Bookkeeper2

Weitere ähnliche Inhalte

Was ist angesagt?

Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Till Rohrmann
 
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward
 
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...Flink Forward
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward
 
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasVirtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasFlink Forward
 
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingApache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingTimo Walther
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingFlink Forward
 
Stateful Distributed Stream Processing
Stateful Distributed Stream ProcessingStateful Distributed Stream Processing
Stateful Distributed Stream ProcessingGyula Fóra
 
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...Flink Forward
 
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...Flink Forward
 
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward
 
Stateful stream processing with Apache Flink
Stateful stream processing with Apache FlinkStateful stream processing with Apache Flink
Stateful stream processing with Apache FlinkKnoldus Inc.
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkFlink Forward
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...Flink Forward
 
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Flink Forward
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward
 
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward
 
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords Stephan Ewen
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Stephan Ewen
 

Was ist angesagt? (20)

Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
 
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
 
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...
Flink Forward Berlin 2017: Pramod Bhatotia, Do Le Quoc - StreamApprox: Approx...
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
 
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasVirtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
 
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingApache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream Processing
 
Stateful Distributed Stream Processing
Stateful Distributed Stream ProcessingStateful Distributed Stream Processing
Stateful Distributed Stream Processing
 
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...
Flink Forward Berlin 2017: Dongwon Kim - Predictive Maintenance with Apache F...
 
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...
Flink Forward Berlin 2017: Steffen Hausmann - Build a Real-time Stream Proces...
 
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
 
Stateful stream processing with Apache Flink
Stateful stream processing with Apache FlinkStateful stream processing with Apache Flink
Stateful stream processing with Apache Flink
 
The Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache FlinkThe Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache Flink
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
 
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
 
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
 
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016
 

Ähnlich wie Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Reimagined for Streaming World

Beam me up, Samza!
Beam me up, Samza!Beam me up, Samza!
Beam me up, Samza!Xinyu Liu
 
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteAdvanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteStreamNative
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016 A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016 Databricks
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streamingdatamantra
 
Fluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconFluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconN Masahiro
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using KafkaKnoldus Inc.
 
Software architecture for data applications
Software architecture for data applicationsSoftware architecture for data applications
Software architecture for data applicationsDing Li
 
February 2017 HUG: Exactly-once end-to-end processing with Apache Apex
February 2017 HUG: Exactly-once end-to-end processing with Apache ApexFebruary 2017 HUG: Exactly-once end-to-end processing with Apache Apex
February 2017 HUG: Exactly-once end-to-end processing with Apache ApexYahoo Developer Network
 
Flink Forward San Francisco 2018 keynote: Srikanth Satya - "Stream Processin...
Flink Forward San Francisco 2018 keynote:  Srikanth Satya - "Stream Processin...Flink Forward San Francisco 2018 keynote:  Srikanth Satya - "Stream Processin...
Flink Forward San Francisco 2018 keynote: Srikanth Satya - "Stream Processin...Flink Forward
 
Kafka streams decoupling with stores
Kafka streams decoupling with storesKafka streams decoupling with stores
Kafka streams decoupling with storesYoni Farin
 
Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flinkRenato Guimaraes
 
A Unified Platform for Real-time Storage and Processing
A Unified Platform for Real-time Storage and ProcessingA Unified Platform for Real-time Storage and Processing
A Unified Platform for Real-time Storage and ProcessingStreamNative
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2aspyker
 
Architectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark StreamingArchitectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark StreamingApache Apex
 
CS8091_BDA_Unit_IV_Stream_Computing
CS8091_BDA_Unit_IV_Stream_ComputingCS8091_BDA_Unit_IV_Stream_Computing
CS8091_BDA_Unit_IV_Stream_ComputingPalani Kumar
 
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
Structured Streaming with Kafka
Structured Streaming with KafkaStructured Streaming with Kafka
Structured Streaming with Kafkadatamantra
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streamingphanleson
 

Ähnlich wie Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Reimagined for Streaming World (20)

Beam me up, Samza!
Beam me up, Samza!Beam me up, Samza!
Beam me up, Samza!
 
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteAdvanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016 A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streaming
 
Fluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconFluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at Kubecon
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using Kafka
 
Software architecture for data applications
Software architecture for data applicationsSoftware architecture for data applications
Software architecture for data applications
 
February 2017 HUG: Exactly-once end-to-end processing with Apache Apex
February 2017 HUG: Exactly-once end-to-end processing with Apache ApexFebruary 2017 HUG: Exactly-once end-to-end processing with Apache Apex
February 2017 HUG: Exactly-once end-to-end processing with Apache Apex
 
Data streaming fundamentals
Data streaming fundamentalsData streaming fundamentals
Data streaming fundamentals
 
Flink Forward San Francisco 2018 keynote: Srikanth Satya - "Stream Processin...
Flink Forward San Francisco 2018 keynote:  Srikanth Satya - "Stream Processin...Flink Forward San Francisco 2018 keynote:  Srikanth Satya - "Stream Processin...
Flink Forward San Francisco 2018 keynote: Srikanth Satya - "Stream Processin...
 
Kafka streams decoupling with stores
Kafka streams decoupling with storesKafka streams decoupling with stores
Kafka streams decoupling with stores
 
Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flink
 
A Unified Platform for Real-time Storage and Processing
A Unified Platform for Real-time Storage and ProcessingA Unified Platform for Real-time Storage and Processing
A Unified Platform for Real-time Storage and Processing
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2
 
Architectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark StreamingArchitectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark Streaming
 
CS8091_BDA_Unit_IV_Stream_Computing
CS8091_BDA_Unit_IV_Stream_ComputingCS8091_BDA_Unit_IV_Stream_Computing
CS8091_BDA_Unit_IV_Stream_Computing
 
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
 
Structured Streaming with Kafka
Structured Streaming with KafkaStructured Streaming with Kafka
Structured Streaming with Kafka
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streaming
 

Mehr von Flink Forward

Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Flink Forward
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkFlink Forward
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...Flink Forward
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Flink Forward
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorFlink Forward
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink Forward
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Flink Forward
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkFlink Forward
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxFlink Forward
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink Forward
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraFlink Forward
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkFlink Forward
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentFlink Forward
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022Flink Forward
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink Forward
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsFlink Forward
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotFlink Forward
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesFlink Forward
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Flink Forward
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergFlink Forward
 

Mehr von Flink Forward (20)

Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes Operator
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async Sink
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at Pinterest
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in Flink
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production Deployment
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easy
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data Alerts
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
 

Kürzlich hochgeladen

Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz1
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...shambhavirathore45
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Delhi Call girls
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...amitlee9823
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...shivangimorya083
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023ymrp368
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...amitlee9823
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightDelhi Call girls
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxJohnnyPlasten
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130Suhani Kapoor
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxfirstjob4
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% SecurePooja Nehwal
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxolyaivanovalion
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Online
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service OnlineCALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Online
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Onlineanilsa9823
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxolyaivanovalion
 

Kürzlich hochgeladen (20)

Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signals
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
 
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in  KishangarhDelhi 99530 vip 56974 Genuine Escort Service Call Girls in  Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptx
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptx
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptx
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Online
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service OnlineCALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Online
CALL ON ➥8923113531 🔝Call Girls Chinhat Lucknow best sexual service Online
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptx
 

Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Reimagined for Streaming World

  • 1. Pravega Storage Reimagined for a Streaming World Srikanth Satya & Tom Kaitchuck Dell EMC Unstructured Storage srikanth.satya@emc.com tom.kaitchuck@emc.com
  • 2. Streaming is Disruptive How do you shrink to zero the time it takes to turn massive volumes of data into information and action? No problem! Just process data as it arrives, quickly, and in a continuous and infinite fashion Demands disruptive systems capabilities ▪ Ability to treat data as continuous and infinite ▪ Ability to dynamically scale ingest, storage, and processing in coordination with data arrival volume ▪ Ability to deliver accurate results processing continuously even with late arriving or out of order data Stateful processors born for streaming, like Apache Flink, are disrupting how we think about data computing … We think the world needs a complementary technology … to similarly disrupt storage.
  • 3. Introducing Pravega Streams A new storage abstraction – a stream – for continuous and infinite data ▪ Named, durable, append-only, infinite sequence of bytes ▪ With low-latency appends to and reads from the tail of the sequence ▪ With high-throughput reads for older portions of the sequence Coordinated scaling of stream storage and stream processing ▪ Stream writes partitioned by app key ▪ Stream reads independently and automatically partitioned by arrival rate SLO ▪ Scaling protocol to allow stream processors to scale in lockstep with storage Enabling system-wide exactly once processing across multiple apps ▪ Streams are ordered and strongly consistent ▪ Chain independent streaming apps via streams ▪ Stream transactions integrate with checkpoint schemes such as the one used in Flink
  • 4. In Place of All This … Stateless Real-Time Processing Batch Processing Messaging Storage Analytics Storage Fast, Approximate Slow, Accurate Archive Storage New Data Old Data
  • 5. … Just Do This! Streaming Storage Fast, Accurate Stateful Stream Processing New & Old Data Pravega Flink Each component in the combined system – writers, streams, readers, apps – is independently, elastically, and dynamically scalable in coordination with data volume arrival rate over time. Sweet!
  • 6. Pravega Streams + Flink First Flink App Worker WorkerRaw Stream … Worker … Segment Protocol coordination between streaming storage and stream processor to dynamically scale up and down the number of segments and Flink workers based on load variance over time Flink utilizes a stream’s transactional writes to extend exactly once processing across multiple, chained apps Social,IoT,… Writers Cooked Stream Next Flink App Sink Segment Segment Sink … Writers scale based on app configuration; stream storage elastically and independently scales based on aggregate incoming volume of data WorkerSegment WorkerSegment …
  • 7. And It’s Just the Beginning … Ingest Buffer & Pub/Sub Persistent Data Structures Search Streaming Analytics Cloud-Scale Storage Pravega Streams Enabling a new generation of distributed middleware reimagined as streaming infrastructure
  • 9. Pravega Architecture Goals • All data is durable – Data is replicated and persisted to disk before being acknowledged • Strict ordering guarantees and exactly once semantics – Across both tail and catch-up reads – Client tracks read offset, Producers use transactions • Lightweight, elastic, infinite, high performance – Support tens of millions of streams – Low (<10ms) latency writes; throughput bounded by network bandwidth – Read pattern (e.g. many catch-up reads) doesn’t affect write performance • Dynamic partitioning of streams based on load and throughput SLO • Capacity is not bounded by the size of a single node
  • 10. Streaming model • Fundamental data structure is an ordered sequence of bytes • Think of it as a durable socket or Unix pipe • Bytes are not interpreted server side • This implicitly guarantees order and non-duplication • Higher layers impose further structure, e.g. message boundaries M5 M4 M3 M2 M1 Pravega Stream
  • 11. Cartoon API public interface SegmentWriter { /** Asynchronously and atomically write data */ void write(ByteBuffer data); /** Asynchronously and atomically write the data if it can be written at the provided offset */ void write(ByteBuffer data, long atOffset); /** Asynchronously and atomically write all of the data from the provided input stream */ void write(InputStream in); } public interface SegmentReader { long fetchCurrentLength(); /** Returns the current offset */ long getOffset(); /** Sets the next offset to read from */ void setOffset(long offset); /** Read bytes from the current offset */ ByteBuffer read(int length); }
  • 12. Idempotent Append Writer Append {0101011} and Assign appendNumber = 7 Pravega Stream
  • 13. Idempotent Append Writer What is appendNumber? 7 Pravega Stream
  • 14. Idempotent Append Writer Append {0101011} and Assign: appendNumber = 7 If and only if appendNumber < 7 Pravega Stream
  • 17. Commodity Server Pravega Bookkeeper Cache SSD SSD Architecture overview - Write Client write(data1) write(data2) Commodity Server Pravega Bookkeeper Cache SSD SSD Commodity Server Bookkeeper Pravega Cache SSD SSD HDFS
  • 18. Commodity Server Pravega Bookkeeper Cache SSD SSD Architecture overview - Read Client read() Commodity Server Pravega Bookkeeper Cache SSD SSD Commodity Server Bookkeeper Pravega Cache SSD SSD HDFS
  • 19. HDFS Commodity Server Pravega Bookkeeper Architecture overview - Evict Commodity Server Pravega Bookkeeper Commodity Server Bookkeeper Pravega • Files in HDFS are organized by Stream Segment • Read-ahead cache optimizations are employed
  • 20. HDFS Commodity Server Pravega Bookkeeper Cache SSD SSD Architecture overview - Read Client read() Commodity Server Pravega Bookkeeper Cache SSD SSD Commodity Server Bookkeeper Pravega Cache SSD SSD
  • 21. Commodity Server Pravega Bookkeeper Architecture overview - Recover Commodity Server Pravega Bookkeeper Commodity Server Bookkeeper Pravega • Data is read from Bookkeeper only in the case of node failure • Used to reconstitute the cache on the remaining hosts HDFS
  • 22. Performance Characteristics • Fast appends to Bookkeeper – Data is persisted durably to disk 3x replicated consistently <10ms • Big block writes to HDFS – Data is mostly cold so it can be erasure encoded and stored cheaply – If data is read, the job is likely a backfill so we can use a large read-ahead • A stream’s capacity is not limited by the capacity of a single machine • Throughput shouldn’t be either …
  • 23. Scaling: Segment Splitting & Merging Segment 7 Segment 4 Segment 3 Segment 1 Segment 2 Segment 0 Time 3 2 1 0 t0 t1 t2 t3 Segment 6 Segment 5 t4 Data Keys Stream S
  • 24. Pravega ProducersProducers Pravega Writers Writer Configuration ka .. kf ➔ S Stream S Stream Segments ss0 ss1 ssn … ProducersProducers Pravega Readers Scaling: Write Parallelism Number of stream segments dynamically changes based on load and SLO1 Segments are split and merged dynamically without manual intervention 2 Writer configurations do not change when segments are split or merged 3
  • 25. EventWriter API /** A writer can write events to a stream. */ public interface EventStreamWriter { /** Send an event to the stream. Event must appear in the stream exactly once */ AckFuture writeEvent(String routingKey, Type event); /** Start a new transaction on this stream */ Transaction<Type> beginTxn(long transactionTimeout); }
  • 26. Pravega ProducersProducers Pravega Writers Writer Configuration ka .. kf ➔ S Stream S Stream Segments ss0 ss1 ssn … ProducersProducers Pravega Readers Scaling: Read Parallelism Readers are notified when segments are split or merged enabling reader parallelism to scale in response to the stream scaling
  • 27. EventReader API public interface EventStreamReader<T> extends AutoCloseable { /** Read the next event from the stream, blocking for up to timeout */ EventRead<T> readNextEvent(long timeout); /** * Close the reader. The segments owned by this reader will automatically be * redistributed to the other readers in the group. */ void close() }
  • 28. Conditional Append Writer Append {0101011} If and only if it would be appended at offset 123 Pravega Stream Writer
  • 29. Synchronizer API /** A means to synchronize state between many processes */ public interface StateSynchronizer<StateT> { /** Gets the state object currently held in memory */ StateT getState(); /** Fetch and apply all updates to bring the local state object up to date */ void fetchUpdates(); /** Creates a new update for the latest state object and applies it atomically */ void updateState(Function<StateT, Update<StateT>> updateGenerator); }
  • 32. EventWriter and Transaction API /** A writer can write events to a stream. */ public interface EventStreamWriter { /** Send an event to the stream. Event must appear in the stream exactly once */ AckFuture writeEvent(String routingKey, Type event); /** Start a new transaction on this stream */ Transaction<Type> beginTxn(long transactionTimeout); } public interface Transaction<Type> { void writeEvent(String routingKey, Type event) throws TxnFailedException; void commit() throws TxnFailedException; void abort(); }
  • 33. Transactions New Item Stream 1, Segment 1 New Item Stream 1, Segment 1, TX-230 New Item …
  • 34. Transactions Writer Append {110…101} to Segment-1-txn-1 Pravega Append {010…111} to Segment-2-txn-1 Pravega Controller Create txn-1 Commit txn-1 Commit txn-1
  • 39. Pravega: Streaming Storage for All ▪ Pravega: an open source project with an open community ▪ To be launched @ Dell EMC World this May 10th ▪ Includes infinite byte stream primitive ▪ Plus an Ingest Buffer with Pub/Sub built on top of streams ▪ And Flink integration! ▪ Visit the Dell EMC booth here @ Flink Forward to learn more ▪ Contact us at pravega@emc.com for even more information!
  • 40. BB-8 Drawing ➢ Stop by the Dell EMC booth and enter to win ➢ Winner will be chosen after the closing Keynote ➢ Must be present to win Pravega Email Pravega@emc.com for the latest news and information on Pravega!
  • 41.
  • 42.
  • 43. Why a new storage system? Source Sinkconnect Sink Sink connect Sink connect
  • 44. Why a new storage system? Connector Real Time Exactly once Durability Storage Capacity Notes HDFS No Yes Yes Years Kafka Yes Source only Yes* (Flushed but not synced) Days Writes are replicated but may not persisted to durable media. (flush.messages=1 bounds this but is not recommended) RabbitMQ Yes Source only Yes* (slowly) Days Durability can be added with a performance hit Cassandra No Yes* (If updates are idempotent) Yes Years App developers need to write custom logic to handle duplicate writes. Sockets Yes No No None
  • 45. Flink storage needs Flink Implications for storage Guarantee Exactly once Exactly once, consistency Latency Very Low Low latency writes (<10ms) Throughput High High throughput Computation model Streaming Streaming model Overhead of fault tolerance mechanism Low Fast recovery Long retention Flow control Natural Data can backlog Capacity not bounded by single host Separation of application logic from fault tolerance Yes Re-reading data provides consistent results License Apache 2.0 Open Source and linkable
  • 46. Shared config public class SharedConfig<K extends Serializable, V extends Serializable> { public V getProperty(K key); public V putPropertyIfAbsent(K key, V value); public boolean removeProperty(K key, V oldValue); public boolean replaceProperty(K key, V oldValue, V newValue); }
  • 47. Smart Workload Distribution Pravega Segment Container Segment Container Segment Container ss1ss0 ss2 ss3 ss4 The hot segment is automatically “split,” and the “child” segments are re-distributed across the cluster relieving the hot spot while maximizing utilization of the cluster’s available IOPs capacity
  • 48. Streaming Storage System Architecture Stream Abstraction Pravega Streaming Service Cloud Scale Storage (HDFS) • High-Throughput • High-Scale, Low-Cost Low-Latency Storage Apache Bookkeeper Auto-Tiering Cache (Rocks) Messaging Apps Real-Time / Batch / Interactive Predictive Analytics Stream Processors: Spark, Flink, … Other Apps & Middleware Pravega Design Innovations 1. Zero-Touch Dynamic Scaling - Automatically scale read/write parallelism based on load and SLO - No service interruptions - No manual reconfiguration of clients - No manual reconfiguration of service resources 2. Smart Workload Distribution - No need to over-provision servers for peak load 3. I/O Path Isolation - For tail writes - For tail reads - For catch-up reads 4. Tiering for “Infinite Streams” 5. Transactions For “Exactly Once”
  • 49. Stream Processor App State App Logic Worker Worker Pravega Optimizations for Stream Processors Input Stream (Pravega) … Worker … Segment Memory-Speed Storage Dynamically split input stream into parallel logs: infinite sequence, low- latency, durable, re-playable with auto- tiering from hot to cold storage. 1 Coordinate via protocol between streaming storage and streaming engine to systematically scale up and down the number of logs and source workers based on load variance over time 2 Support streaming write COMMIT operation to extend Exactly Once processing semantics across multiple, chained applications 3 Social,IoT Producers Output Stream (Pravega) Stream Processor 2nd App SegmentSink Segment Segment
  • 50. Comparing Pravega and Kafka Design Points Unlike Kafka, Pravega is designed to be a durable and permanent storage system Quality Pravega Goal Kafka Design Point Data Durability Replicated and persisted to disk before ACK Replicated but not persisted to disk before ACK Strict Ordering Consistent ordering on tail and catch-up reads Messages may get reordered Exactly Once Producers can use transactions for atomicity Messages may get duplicated Scale Tens of millions of streams per cluster Thousands of topics per cluster Elastic Dynamic partitioning of streams based on load and SLO Statically configured partitions Size Log size is not bounded by the capacity of any single node Partition size is bounded by capacity of filesystem on its hosting node Transparently migrate/retrieve data from Tier 2 storage for older parts of the log External ETL required to move data to Tier 2 storage; no access to data via Kafka once moved Performance Low (<10ms) latency durable writes; throughput bounded by network bandwidth Low-latency achieved only by reducing replication/ reliability parameters Read pattern (e.g. many catch-up readers) does not affect write performance Read patterns adversely affects write performance due to reliance on OS filesystem cache ✗ ✗ ✗ ✗ ✗ ✗ ✗ ✗ ✗
  • 51. Attributes Connector Streaming Exactly once Durability Storage Capacity HDFS No Yes Yes Years Kafka Yes Source only Yes* (Flushed but not synced) Days Pravega Yes:
 Byte oriented and event oriented Yes.
 With either idempotent producers, or transactions Yes. Always flushed and synced, with low latency. As much as you can fit in your HDFS cluster.
  • 52. Apache BookKeeper Architecture overview - Write Metadata
 + Cache Check Metadata 1 Record data to Log 2 Update 
 Metadata / cache3ACK 4
  • 53. Architecture overview - Read Metadata
 + Cache Apache BookKeeper Pull from cache 2.a Check Metadata 1 Pull from HDFS 2.b
  • 54. Architecture overview - Evict Metadata
 + Cache Apache BookKeeper Write contents from cache to HDFS 1 Mark data for removal 2
  • 55. Architecture overview - Recover Metadata
 + Cache Apache BookKeeper Take ownership of BK Ledger 1 Re-populate metadata/cache from Bookkeeper2