SlideShare una empresa de Scribd logo
1 de 39
1
Reliability Guarantees in
Apache Kafka
Gwen Shapira, Product Manager
@gwenshap
2
Streaming Platform
Producer Consumer
Streaming Applications
Connectors Connectors
Apache
Kafka
3
Versions of Apache Kafka
• 0.7.0 <- Please don’t
• 0.8.0 <- Replication exists, it will continue evolving with every release
• 0.8.2 <- New producer, offset commits to Kafka
• 0.9.0 <- New consumer, Connect APIs
• 0.10.0 <- New consumer improvements, Streams APIs
• 0.11.0 <- Idempotent producer, transactional semantics, Exactly once.
• Future <- Out-of-the-box reliable configuration:
https://issues.apache.org/jira/browse/KAFKA-5795
4
If Kafka is a critical piece of our pipeline
§ Can we be 100% sure that our data will get there?
§ Can we lose messages?
§ How do we verify?
§ Who’s fault is it?
5
Distributed Systems
§ Things Fail
§ Systems are designed to
tolerate failure
§ We must expect failures
and design our code and
configure our systems to
handle them
6
Network
Broker MachineClient Machine
Data Flow
Kafka Client
Broker
O/S Socket Buffer
NIC
NIC
Page Cache
Disk
Application
Thread
O/S Socket Buffercallback
✗
✗
✗
✗
✗
✗
✗✗ data
ack / exception
Replication
7
Kafka is super reliable.
… if you know how to configure it
that way.
8
Replication is your friend
§ Kafka protects against failures by replicating data
§ The unit of replication is the partition
§ One replica is designated as the Leader
§ Follower replicas fetch data from the leader
§ The leader holds the list of “in-sync” replicas
9
Replication and ISRs
0
1
2
0
1
2
0
1
2
Producer
Broker 100 Broker 101 Broker 102
Topic:
Partitions:
Replicas:
my_topic
3
3
Partition:
Leader:
ISR:
1
101
100,102
Partition:
Leader:
ISR:
2
102
101,100
Partition:
Leader:
ISR:
0
100
101,102
10
ISR
2 things make a replica in-sync
• replica.lag.time.max.ms – replica that didn t fetch or is behind
• Connection to Zookeeper
11
Terminology
Acked
• Producers will not retry sending.
• Depends on producer setting
Committed
• Only when message got to all ISR
(future leaders have it).
• Consumers can read.
• replica.lag.time.max.ms controls: how long can a
dead replica prevent consumers from reading?
Committed Offsets
• Consumer told Kafka the latest offsets it read. By
default the consumer will not see these events
again.
12
Replication
Acks = all
• Waits for all in-sync replicas to reply.
Replica 3
100
Replica 2
100
Replica 1
100
Time
13
Replica 3 stopped replicating for some reason
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101
Time
Acked in acks =
all
“committed”
Acked in acks = 1
but not
“committed”
14
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101
Time
One replica drops out of ISR, or goes offline
All messages are now acked and committed
15
2nd Replica drops out, or is offline
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101
102
103
104Time
16
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101
102
103
104Time
Now we’re in trouble
✗
17
Replication
If Replica 2 or 3 come back online before the leader, you can will lose data.
Replica 3
100
Replica 2
100
101
Replica 1
100
101
102
103
104
All those are
“acked” and
“committed”
Time
18
So what to do
Disable Unclean Leader Election
• unclean.leader.election.enable = false
• Default from 0.11.0
Set replication factor
• default.replication.factor = 3
Set minimum ISRs
• min.insync.replicas = 2
19
Replication
Replication = 3
Min ISR = 2
Replica 3
100
Replica 2
100
Replica 1
100
Time
20
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101
Time
One replica drops out of ISR, or goes offline
21
Replication
Replica 3
100
Replica 2
100
101
Replica 1
100
101102
103
104
Time
2nd Replica fails out, or is out of sync
Buffers in
Producer
22
23
Producer Internals
Producer sends batches of messages to a buffer
M3
Application
Thread
Application
Thread
Application
Thread
send()
M2 M1 M0
Batch 3
Batch 2
Batch 1
Fail
?
response
retry
Update
Future
callback
drain
Metadata or
Exception
24
Basics
• Durability: request.required.acks
• 0 The message is written to the network (buffer)
• 1 The message is written to the leader
• all The producer gets an ack after all ISRs receive the data; the message is committed
• Retries:
• Default is 0.
• How long of downtime I need to survive divide by retry.backoff.ms
• KIP-91 may improve things
• Memory for retries:
• Have plenty of buffer.memory
• max.block.ms = Long.MAX_VALUE
• Or handle the BufferExhaustedException / TimeoutException yourself
• In flight requests could lead to message re-ordering
• Don’t forget to close the producer: producer.close()will block until in-flight txns complete
25
“New” Producer
All calls are non-blocking async
3 Options for checking for failures:
• Don’t. Just call send() and YOLO!
• Immediately block for response: send().get()
• Do follow-up work in callback (but not retries)
26
27
Consumer
Three choices One good choice for Consumer API:
• Simple Consumer
• High Level Consumer (ZookeeperConsumer)
• New KafkaConsumer
28
New Consumer – auto commit
props.put("enable.auto.commit", "true");
props.put("auto.commit.interval.ms", "10000");
KafkaConsumer<String, String> consumer = new KafkaConsumer<String,
String>(props);
consumer.subscribe(Arrays.asList("foo", "bar"));
while (true) {
ConsumerRecords<String, String> records = consumer.poll(100);
for (ConsumerRecord<String, String> record : records) {
processAndUpdateDB(record);
}
}
What if we crash
after 8 seconds?
Commit automatically
every 10 seconds
29
New Consumer – manual commit
props.put("enable.auto.commit", "false");
KafkaConsumer<String, String> consumer = new KafkaConsumer<String,
String>(props);
consumer.subscribe(Arrays.asList("foo", "bar"));
while (true) {
ConsumerRecords<String, String> records = consumer.poll(100);
for (ConsumerRecord<String, String> record : records)
processAndUpdateDB(record);
consumer.commitSync();
}
Commit entire
batch outside the
loop!
30
Rebalances Happen
31
Handling Rebalances
private class HandleRebalance implements ConsumerRebalanceListener {
public void onPartitionsAssigned(Collection<TopicPartition> tp) {
}
public void onPartitionsRevoked(Collection<TopicPartition> tp) {
System.out.println("Lost partitions in rebalance.
Committing current offsets:" +
currentOffsets);
consumer.commitSync(currentOffsets);
}
}
32
Minimize Duplicates for At Least Once Consuming
1. Commit your own offsets
autocommit.enable = false
2. Use Rebalance Listener
3. Commit only what you are done
processing
33
Exactly Once Semantics
• At most once is easy
• At least once is not bad either – commit after 100% sure data is safe
• Exactly once is tricky
• Commit data and offsets in one transaction
• Idempotent producer
• Kafka Connect:
• many connectors (especially Confluent’s) are exactly once
• by using an external database to write events and store offsets in one transaction
• Kafka Streams:
• starting at 0.11.0 have easy to configure exactly once (exactly.once=true).
• Other stream processing systems – have their own thing.
34
How we test Kafka?
"""Replication tests.
These tests verify that replication provides simple durability guarantees by checking that data acked by
brokers is still available for consumption in the face of various failure scenarios.
Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2
- Produce messages in the background
- Consume messages in the background
- Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9)
- When done driving failures, stop producing, and finish consuming
- Validate that every acked message was consumed
"""
35
Monitoring for Data Loss
36
And
catching
duplicates
too
37
Monitoring for Data Loss
• Monitor for producer errors – watch the retry numbers
• Monitor consumer lag – MaxLag or via offsets
• Each message contains a CreateTime timestamp
• Each producer can report message counts and offsets to a special topic
• Each consumer reports message counts to another special topic
• Reconcile the results
38
Be Safe, Not Sorry
Acks = all
Max.block.ms = Long.MAX_VALUE
Retries = MAX_INT
( Max.inflight.requests.per.connection = 1 )
Producer.close()
Replication-factor >= 3
Min.insync.replicas = 2
Unclean.leader.election = false
Auto.offset.commit = false
Commit after processing
Monitor!
39
Thank You!

Más contenido relacionado

La actualidad más candente

Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explainedconfluent
 
From Zero to Hero with Kafka Connect
From Zero to Hero with Kafka ConnectFrom Zero to Hero with Kafka Connect
From Zero to Hero with Kafka Connectconfluent
 
Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?confluent
 
A Deep Dive into Kafka Controller
A Deep Dive into Kafka ControllerA Deep Dive into Kafka Controller
A Deep Dive into Kafka Controllerconfluent
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache KafkaChhavi Parasher
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafkaemreakis
 
Developing Real-Time Data Pipelines with Apache Kafka
Developing Real-Time Data Pipelines with Apache KafkaDeveloping Real-Time Data Pipelines with Apache Kafka
Developing Real-Time Data Pipelines with Apache KafkaJoe Stein
 
Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...
 Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra... Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...
Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...HostedbyConfluent
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka StreamsGuozhang Wang
 
Kafka Retry and DLQ
Kafka Retry and DLQKafka Retry and DLQ
Kafka Retry and DLQGeorge Teo
 
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity PlanningFrom Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planningconfluent
 
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Jean-Paul Azar
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...DataWorks Summit/Hadoop Summit
 
ksqlDB - Stream Processing simplified!
ksqlDB - Stream Processing simplified!ksqlDB - Stream Processing simplified!
ksqlDB - Stream Processing simplified!Guido Schmutz
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database Systemconfluent
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Storesconfluent
 

La actualidad más candente (20)

Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explained
 
From Zero to Hero with Kafka Connect
From Zero to Hero with Kafka ConnectFrom Zero to Hero with Kafka Connect
From Zero to Hero with Kafka Connect
 
Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?
 
A Deep Dive into Kafka Controller
A Deep Dive into Kafka ControllerA Deep Dive into Kafka Controller
A Deep Dive into Kafka Controller
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache Kafka
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
Developing Real-Time Data Pipelines with Apache Kafka
Developing Real-Time Data Pipelines with Apache KafkaDeveloping Real-Time Data Pipelines with Apache Kafka
Developing Real-Time Data Pipelines with Apache Kafka
 
kafka
kafkakafka
kafka
 
Apache kafka
Apache kafkaApache kafka
Apache kafka
 
Apache kafka
Apache kafkaApache kafka
Apache kafka
 
Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...
 Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra... Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...
Disaster Recovery Options Running Apache Kafka in Kubernetes with Rema Subra...
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
Kafka Retry and DLQ
Kafka Retry and DLQKafka Retry and DLQ
Kafka Retry and DLQ
 
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity PlanningFrom Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
 
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
 
ksqlDB - Stream Processing simplified!
ksqlDB - Stream Processing simplified!ksqlDB - Stream Processing simplified!
ksqlDB - Stream Processing simplified!
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Stores
 

Destacado

Metrics Are Not Enough: Monitoring Apache Kafka and Streaming Applications
Metrics Are Not Enough: Monitoring Apache Kafka and Streaming ApplicationsMetrics Are Not Enough: Monitoring Apache Kafka and Streaming Applications
Metrics Are Not Enough: Monitoring Apache Kafka and Streaming Applicationsconfluent
 
Putting the Micro into Microservices with Stateful Stream Processing
Putting the Micro into Microservices with Stateful Stream ProcessingPutting the Micro into Microservices with Stateful Stream Processing
Putting the Micro into Microservices with Stateful Stream Processingconfluent
 
Streaming Data and Stream Processing with Apache Kafka
Streaming Data and Stream Processing with Apache KafkaStreaming Data and Stream Processing with Apache Kafka
Streaming Data and Stream Processing with Apache Kafkaconfluent
 
Real-world Streaming Architectures
Real-world Streaming ArchitecturesReal-world Streaming Architectures
Real-world Streaming Architecturesconfluent
 
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 PeopleKafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 Peopleconfluent
 
Building Microservices with Apache Kafka
Building Microservices with Apache KafkaBuilding Microservices with Apache Kafka
Building Microservices with Apache Kafkaconfluent
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...confluent
 
Disaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache KafkaDisaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache Kafkaconfluent
 
Common Patterns of Multi Data-Center Architectures with Apache Kafka
Common Patterns of Multi Data-Center Architectures with Apache KafkaCommon Patterns of Multi Data-Center Architectures with Apache Kafka
Common Patterns of Multi Data-Center Architectures with Apache Kafkaconfluent
 
Exactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache KafkaExactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache Kafkaconfluent
 

Destacado (10)

Metrics Are Not Enough: Monitoring Apache Kafka and Streaming Applications
Metrics Are Not Enough: Monitoring Apache Kafka and Streaming ApplicationsMetrics Are Not Enough: Monitoring Apache Kafka and Streaming Applications
Metrics Are Not Enough: Monitoring Apache Kafka and Streaming Applications
 
Putting the Micro into Microservices with Stateful Stream Processing
Putting the Micro into Microservices with Stateful Stream ProcessingPutting the Micro into Microservices with Stateful Stream Processing
Putting the Micro into Microservices with Stateful Stream Processing
 
Streaming Data and Stream Processing with Apache Kafka
Streaming Data and Stream Processing with Apache KafkaStreaming Data and Stream Processing with Apache Kafka
Streaming Data and Stream Processing with Apache Kafka
 
Real-world Streaming Architectures
Real-world Streaming ArchitecturesReal-world Streaming Architectures
Real-world Streaming Architectures
 
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 PeopleKafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
 
Building Microservices with Apache Kafka
Building Microservices with Apache KafkaBuilding Microservices with Apache Kafka
Building Microservices with Apache Kafka
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
 
Disaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache KafkaDisaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache Kafka
 
Common Patterns of Multi Data-Center Architectures with Apache Kafka
Common Patterns of Multi Data-Center Architectures with Apache KafkaCommon Patterns of Multi Data-Center Architectures with Apache Kafka
Common Patterns of Multi Data-Center Architectures with Apache Kafka
 
Exactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache KafkaExactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache Kafka
 

Similar a Reliability Guarantees in Apache Kafka: How to Configure for 100% Durability

Apache Kafka Reliability
Apache Kafka Reliability Apache Kafka Reliability
Apache Kafka Reliability Jeff Holoman
 
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...confluent
 
Kafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereKafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereGwen (Chen) Shapira
 
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015 Apache Kafka Reliability Guarantees StrataHadoop NYC 2015
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015 Jeff Holoman
 
Seek and Destroy Kafka Under Replication
Seek and Destroy Kafka Under ReplicationSeek and Destroy Kafka Under Replication
Seek and Destroy Kafka Under ReplicationHostedbyConfluent
 
Common issues with Apache Kafka® Producer
Common issues with Apache Kafka® ProducerCommon issues with Apache Kafka® Producer
Common issues with Apache Kafka® Producerconfluent
 
Kafka Reliability Guarantees ATL Kafka User Group
Kafka Reliability Guarantees ATL Kafka User GroupKafka Reliability Guarantees ATL Kafka User Group
Kafka Reliability Guarantees ATL Kafka User GroupJeff Holoman
 
Webinar patterns anti patterns
Webinar patterns anti patternsWebinar patterns anti patterns
Webinar patterns anti patternsconfluent
 
Apache Kafka
Apache KafkaApache Kafka
Apache KafkaJoe Stein
 
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018Otávio Carvalho
 
World of Tanks Experience of Using Kafka
World of Tanks Experience of Using KafkaWorld of Tanks Experience of Using Kafka
World of Tanks Experience of Using KafkaLevon Avakyan
 
Reactive solutions using java 9 and spring reactor
Reactive solutions using java 9 and spring reactorReactive solutions using java 9 and spring reactor
Reactive solutions using java 9 and spring reactorOrenEzer1
 
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and Hadoop
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and HadoopEventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and Hadoop
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and HadoopAyon Sinha
 
SFBigAnalytics_20190724: Monitor kafka like a Pro
SFBigAnalytics_20190724: Monitor kafka like a ProSFBigAnalytics_20190724: Monitor kafka like a Pro
SFBigAnalytics_20190724: Monitor kafka like a ProChester Chen
 
Oracle real application clusters system tests with demo
Oracle real application clusters system tests with demoOracle real application clusters system tests with demo
Oracle real application clusters system tests with demoAjith Narayanan
 
Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Guido Schmutz
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache KafkaJeff Holoman
 

Similar a Reliability Guarantees in Apache Kafka: How to Configure for 100% Durability (20)

Apache Kafka Reliability
Apache Kafka Reliability Apache Kafka Reliability
Apache Kafka Reliability
 
Kafka reliability velocity 17
Kafka reliability   velocity 17Kafka reliability   velocity 17
Kafka reliability velocity 17
 
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...
When it Absolutely, Positively, Has to be There: Reliability Guarantees in Ka...
 
Kafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereKafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be there
 
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015 Apache Kafka Reliability Guarantees StrataHadoop NYC 2015
Apache Kafka Reliability Guarantees StrataHadoop NYC 2015
 
Seek and Destroy Kafka Under Replication
Seek and Destroy Kafka Under ReplicationSeek and Destroy Kafka Under Replication
Seek and Destroy Kafka Under Replication
 
Common issues with Apache Kafka® Producer
Common issues with Apache Kafka® ProducerCommon issues with Apache Kafka® Producer
Common issues with Apache Kafka® Producer
 
Kafka Reliability Guarantees ATL Kafka User Group
Kafka Reliability Guarantees ATL Kafka User GroupKafka Reliability Guarantees ATL Kafka User Group
Kafka Reliability Guarantees ATL Kafka User Group
 
Webinar patterns anti patterns
Webinar patterns anti patternsWebinar patterns anti patterns
Webinar patterns anti patterns
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018
 
intro-kafka
intro-kafkaintro-kafka
intro-kafka
 
World of Tanks Experience of Using Kafka
World of Tanks Experience of Using KafkaWorld of Tanks Experience of Using Kafka
World of Tanks Experience of Using Kafka
 
Reactive solutions using java 9 and spring reactor
Reactive solutions using java 9 and spring reactorReactive solutions using java 9 and spring reactor
Reactive solutions using java 9 and spring reactor
 
Kafka Explainaton
Kafka ExplainatonKafka Explainaton
Kafka Explainaton
 
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and Hadoop
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and HadoopEventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and Hadoop
Eventual Consistency @WalmartLabs with Kafka, Avro, SolrCloud and Hadoop
 
SFBigAnalytics_20190724: Monitor kafka like a Pro
SFBigAnalytics_20190724: Monitor kafka like a ProSFBigAnalytics_20190724: Monitor kafka like a Pro
SFBigAnalytics_20190724: Monitor kafka like a Pro
 
Oracle real application clusters system tests with demo
Oracle real application clusters system tests with demoOracle real application clusters system tests with demo
Oracle real application clusters system tests with demo
 
Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
 

Más de confluent

Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...confluent
 
Santander Stream Processing with Apache Flink
Santander Stream Processing with Apache FlinkSantander Stream Processing with Apache Flink
Santander Stream Processing with Apache Flinkconfluent
 
Unlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insightsUnlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insightsconfluent
 
Workshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con FlinkWorkshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con Flinkconfluent
 
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...confluent
 
AWS Immersion Day Mapfre - Confluent
AWS Immersion Day Mapfre   -   ConfluentAWS Immersion Day Mapfre   -   Confluent
AWS Immersion Day Mapfre - Confluentconfluent
 
Eventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkEventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkconfluent
 
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent CloudQ&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent Cloudconfluent
 
Citi TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep DiveCiti TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep Diveconfluent
 
Build real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with ConfluentBuild real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with Confluentconfluent
 
Q&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service MeshQ&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service Meshconfluent
 
Citi Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka MicroservicesCiti Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka Microservicesconfluent
 
Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3confluent
 
Citi Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging ModernizationCiti Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging Modernizationconfluent
 
Citi Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataCiti Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataconfluent
 
Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2confluent
 
Data In Motion Paris 2023
Data In Motion Paris 2023Data In Motion Paris 2023
Data In Motion Paris 2023confluent
 
Confluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with SynthesisConfluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with Synthesisconfluent
 
The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023confluent
 
The Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data StreamsThe Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data Streamsconfluent
 

Más de confluent (20)

Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
 
Santander Stream Processing with Apache Flink
Santander Stream Processing with Apache FlinkSantander Stream Processing with Apache Flink
Santander Stream Processing with Apache Flink
 
Unlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insightsUnlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insights
 
Workshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con FlinkWorkshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con Flink
 
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
 
AWS Immersion Day Mapfre - Confluent
AWS Immersion Day Mapfre   -   ConfluentAWS Immersion Day Mapfre   -   Confluent
AWS Immersion Day Mapfre - Confluent
 
Eventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkEventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalk
 
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent CloudQ&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
 
Citi TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep DiveCiti TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep Dive
 
Build real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with ConfluentBuild real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with Confluent
 
Q&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service MeshQ&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service Mesh
 
Citi Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka MicroservicesCiti Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka Microservices
 
Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3
 
Citi Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging ModernizationCiti Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging Modernization
 
Citi Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataCiti Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time data
 
Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2
 
Data In Motion Paris 2023
Data In Motion Paris 2023Data In Motion Paris 2023
Data In Motion Paris 2023
 
Confluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with SynthesisConfluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with Synthesis
 
The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023
 
The Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data StreamsThe Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data Streams
 

Último

Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfkalichargn70th171
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsAndolasoft Inc
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...ICS
 
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfThe Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfkalichargn70th171
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxbodapatigopi8531
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comFatema Valibhai
 
Active Directory Penetration Testing, cionsystems.com.pdf
Active Directory Penetration Testing, cionsystems.com.pdfActive Directory Penetration Testing, cionsystems.com.pdf
Active Directory Penetration Testing, cionsystems.com.pdfCionsystems
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsArshad QA
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendArshad QA
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...gurkirankumar98700
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...panagenda
 
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️anilsa9823
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...OnePlan Solutions
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about usDynamic Netsoft
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantAxelRicardoTrocheRiq
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsAlberto González Trastoy
 
5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdfWave PLM
 

Último (20)

Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.js
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
 
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfThe Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptx
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.com
 
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
Active Directory Penetration Testing, cionsystems.com.pdf
Active Directory Penetration Testing, cionsystems.com.pdfActive Directory Penetration Testing, cionsystems.com.pdf
Active Directory Penetration Testing, cionsystems.com.pdf
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview Questions
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and Backend
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
 
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
 
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about us
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service Consultant
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
 
5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf
 

Reliability Guarantees in Apache Kafka: How to Configure for 100% Durability

  • 1. 1 Reliability Guarantees in Apache Kafka Gwen Shapira, Product Manager @gwenshap
  • 2. 2 Streaming Platform Producer Consumer Streaming Applications Connectors Connectors Apache Kafka
  • 3. 3 Versions of Apache Kafka • 0.7.0 <- Please don’t • 0.8.0 <- Replication exists, it will continue evolving with every release • 0.8.2 <- New producer, offset commits to Kafka • 0.9.0 <- New consumer, Connect APIs • 0.10.0 <- New consumer improvements, Streams APIs • 0.11.0 <- Idempotent producer, transactional semantics, Exactly once. • Future <- Out-of-the-box reliable configuration: https://issues.apache.org/jira/browse/KAFKA-5795
  • 4. 4 If Kafka is a critical piece of our pipeline § Can we be 100% sure that our data will get there? § Can we lose messages? § How do we verify? § Who’s fault is it?
  • 5. 5 Distributed Systems § Things Fail § Systems are designed to tolerate failure § We must expect failures and design our code and configure our systems to handle them
  • 6. 6 Network Broker MachineClient Machine Data Flow Kafka Client Broker O/S Socket Buffer NIC NIC Page Cache Disk Application Thread O/S Socket Buffercallback ✗ ✗ ✗ ✗ ✗ ✗ ✗✗ data ack / exception Replication
  • 7. 7 Kafka is super reliable. … if you know how to configure it that way.
  • 8. 8 Replication is your friend § Kafka protects against failures by replicating data § The unit of replication is the partition § One replica is designated as the Leader § Follower replicas fetch data from the leader § The leader holds the list of “in-sync” replicas
  • 9. 9 Replication and ISRs 0 1 2 0 1 2 0 1 2 Producer Broker 100 Broker 101 Broker 102 Topic: Partitions: Replicas: my_topic 3 3 Partition: Leader: ISR: 1 101 100,102 Partition: Leader: ISR: 2 102 101,100 Partition: Leader: ISR: 0 100 101,102
  • 10. 10 ISR 2 things make a replica in-sync • replica.lag.time.max.ms – replica that didn t fetch or is behind • Connection to Zookeeper
  • 11. 11 Terminology Acked • Producers will not retry sending. • Depends on producer setting Committed • Only when message got to all ISR (future leaders have it). • Consumers can read. • replica.lag.time.max.ms controls: how long can a dead replica prevent consumers from reading? Committed Offsets • Consumer told Kafka the latest offsets it read. By default the consumer will not see these events again.
  • 12. 12 Replication Acks = all • Waits for all in-sync replicas to reply. Replica 3 100 Replica 2 100 Replica 1 100 Time
  • 13. 13 Replica 3 stopped replicating for some reason Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101 Time Acked in acks = all “committed” Acked in acks = 1 but not “committed”
  • 14. 14 Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101 Time One replica drops out of ISR, or goes offline All messages are now acked and committed
  • 15. 15 2nd Replica drops out, or is offline Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101 102 103 104Time
  • 16. 16 Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101 102 103 104Time Now we’re in trouble ✗
  • 17. 17 Replication If Replica 2 or 3 come back online before the leader, you can will lose data. Replica 3 100 Replica 2 100 101 Replica 1 100 101 102 103 104 All those are “acked” and “committed” Time
  • 18. 18 So what to do Disable Unclean Leader Election • unclean.leader.election.enable = false • Default from 0.11.0 Set replication factor • default.replication.factor = 3 Set minimum ISRs • min.insync.replicas = 2
  • 19. 19 Replication Replication = 3 Min ISR = 2 Replica 3 100 Replica 2 100 Replica 1 100 Time
  • 20. 20 Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101 Time One replica drops out of ISR, or goes offline
  • 21. 21 Replication Replica 3 100 Replica 2 100 101 Replica 1 100 101102 103 104 Time 2nd Replica fails out, or is out of sync Buffers in Producer
  • 22. 22
  • 23. 23 Producer Internals Producer sends batches of messages to a buffer M3 Application Thread Application Thread Application Thread send() M2 M1 M0 Batch 3 Batch 2 Batch 1 Fail ? response retry Update Future callback drain Metadata or Exception
  • 24. 24 Basics • Durability: request.required.acks • 0 The message is written to the network (buffer) • 1 The message is written to the leader • all The producer gets an ack after all ISRs receive the data; the message is committed • Retries: • Default is 0. • How long of downtime I need to survive divide by retry.backoff.ms • KIP-91 may improve things • Memory for retries: • Have plenty of buffer.memory • max.block.ms = Long.MAX_VALUE • Or handle the BufferExhaustedException / TimeoutException yourself • In flight requests could lead to message re-ordering • Don’t forget to close the producer: producer.close()will block until in-flight txns complete
  • 25. 25 “New” Producer All calls are non-blocking async 3 Options for checking for failures: • Don’t. Just call send() and YOLO! • Immediately block for response: send().get() • Do follow-up work in callback (but not retries)
  • 26. 26
  • 27. 27 Consumer Three choices One good choice for Consumer API: • Simple Consumer • High Level Consumer (ZookeeperConsumer) • New KafkaConsumer
  • 28. 28 New Consumer – auto commit props.put("enable.auto.commit", "true"); props.put("auto.commit.interval.ms", "10000"); KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); consumer.subscribe(Arrays.asList("foo", "bar")); while (true) { ConsumerRecords<String, String> records = consumer.poll(100); for (ConsumerRecord<String, String> record : records) { processAndUpdateDB(record); } } What if we crash after 8 seconds? Commit automatically every 10 seconds
  • 29. 29 New Consumer – manual commit props.put("enable.auto.commit", "false"); KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); consumer.subscribe(Arrays.asList("foo", "bar")); while (true) { ConsumerRecords<String, String> records = consumer.poll(100); for (ConsumerRecord<String, String> record : records) processAndUpdateDB(record); consumer.commitSync(); } Commit entire batch outside the loop!
  • 31. 31 Handling Rebalances private class HandleRebalance implements ConsumerRebalanceListener { public void onPartitionsAssigned(Collection<TopicPartition> tp) { } public void onPartitionsRevoked(Collection<TopicPartition> tp) { System.out.println("Lost partitions in rebalance. Committing current offsets:" + currentOffsets); consumer.commitSync(currentOffsets); } }
  • 32. 32 Minimize Duplicates for At Least Once Consuming 1. Commit your own offsets autocommit.enable = false 2. Use Rebalance Listener 3. Commit only what you are done processing
  • 33. 33 Exactly Once Semantics • At most once is easy • At least once is not bad either – commit after 100% sure data is safe • Exactly once is tricky • Commit data and offsets in one transaction • Idempotent producer • Kafka Connect: • many connectors (especially Confluent’s) are exactly once • by using an external database to write events and store offsets in one transaction • Kafka Streams: • starting at 0.11.0 have easy to configure exactly once (exactly.once=true). • Other stream processing systems – have their own thing.
  • 34. 34 How we test Kafka? """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by brokers is still available for consumption in the face of various failure scenarios. Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2 - Produce messages in the background - Consume messages in the background - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9) - When done driving failures, stop producing, and finish consuming - Validate that every acked message was consumed """
  • 37. 37 Monitoring for Data Loss • Monitor for producer errors – watch the retry numbers • Monitor consumer lag – MaxLag or via offsets • Each message contains a CreateTime timestamp • Each producer can report message counts and offsets to a special topic • Each consumer reports message counts to another special topic • Reconcile the results
  • 38. 38 Be Safe, Not Sorry Acks = all Max.block.ms = Long.MAX_VALUE Retries = MAX_INT ( Max.inflight.requests.per.connection = 1 ) Producer.close() Replication-factor >= 3 Min.insync.replicas = 2 Unclean.leader.election = false Auto.offset.commit = false Commit after processing Monitor!