SlideShare a Scribd company logo
Common issues with Apache
Kafka® Producer
Badai Aqrandista, Senior Technical Support Engineer
Introduction
2
• My name is BADAI AQRANDISTA
• I started as a web developer, building website with Perl
and PHP in 2005.
• Experience supporting applications on Linux/UNIX
environment, from hotel booking engine,
telecommunication billing system, and mining equipment
monitoring system.
• Currently working for Confluent as Senior Technical
Support Engineer.
Kafka in a nutshell
3
• Kafka is a Pub/Sub system
• Kafka Producer sends record into Kafka
broker
• Kafka Consumer fetches record from
Kafka broker
• Kafka broker persists any data it receives
until retention period expires
PRODUCER CONSUME
R
Kafka Producer Internals
Kafka Producer Internals
5
• KafkaProducer API:
• public Future<RecordMetadata> send(ProducerRecord<K,V> record)
• public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback callback)
• KafkaProducer#send method is asynchronous.
• It does not immediately send the record to Kafka broker.
• It puts the record in an internal queue and an internal queue will send multiple records as a
batch.
Batch
Record
Key
Value
Record
Key
Value
Record
Key
Value
Kafka Producer Internals
6
• Each Kafka Producer batch corresponds to a partitions.
• Kafka Producer determines the batch to append a record to based on the record key.
• If record key is “null”, Kafka Producer will choose the batch randomly.
• If record key is not “null”, Kafka Producer will use the hash of the record key to determine
the partition number.
• One or more batches are sent to the Kafka broker in a PRODUCE request.
Kafka Producer Internals
7
• Kafka Producer internal thread sends a batch to Kafka broker based on these
configuration:
• “batch.size” – defaults to 16 kB
• “linger.ms” – defaults to 0
• So, Kafka Producer internal thread sends a batch to Kafka broker when:
• The total size of records in the batch exceeds “batch.size”, or
• The time since batch creation exceeds “linger.ms”, or
• Kafka Producer ”flush()” method is called (directly or indirectly via “close()”).
• Kafka Producer only creates one connection to each broker.
• In the end, every batch for a Kafka broker must be sent sequentially through this one
connection.
• The maximum number of batches sent to each broker at any one time is controlled by
“max.in.flight.requests.per.connection”, which defaults to 5.
Kafka Producer Issues
Kafka Producer Issues
9
1. Failure to connect to Kafka broker
2. Record is too large
3. Batch expires before sending
4. Not enough replicas error
Failure to connect to Kafka broker
10
• This error is not obvious, but it means failure to connect to Kafka broker.
• The error message looks like this:
• [2021-08-02 12:57:44,097] WARN [Producer clientId=producer-1] Connection to node -1
(kafka1/172.20.0.6:9093) could not be established. Broker may not be available.
(org.apache.kafka.clients.NetworkClient)
• How to fix this:
• Check the broker configuration to confirm the listener port and security protocol
• Check the hostname or the IP address of the broker
• Confirm that Kafka Producer’s bootstrap.server configuration is correct
• Confirm that connectivity exists between Kafka Producer’s host and Kafka broker hosts with commands
such as:
• ping {BROKER_HOST}
• nc {BROKER_HOST} {BROKER PORT}
• openssl s_client -connect {BROKER_HOST}:{BROKER_PORT}
Record is too large
11
• This error is because the record size is greater than “max.request.size” configuration, which
defaults to 1048576 (1 MB).
• The error message is like this:
• org.apache.kafka.common.errors.RecordTooLargeException: The message is 1600088 bytes when
serialized which is larger than 1048576, which is the value of the max.request.size configuration.
• How to fix it:
• Reduce the record size. This requires a change in the application that generates the record.
• If you cannot reduce the record size, you can increase producer configuration “max.request.size”. If you
do this, you also need to increase topic configuration “max.message.bytes”.
• Note: “max.request.size” is the maximum request size AFTER serialization but BEFORE
compression. So, setting compression will not fix this.
Batch expires before sending
12
• This error is a symptom of slow transfer time (on network) or slow processing (on Kafka
broker).
• The error looks like this:
• org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for test1-0:1500 ms has
passed since batch creation
• Sanity checks:
• Is the topic partition online? Topic partition is online if one or more Kafka brokers hosting the replicas
are online.
• Use “kafka-topics --bootstrap-server {BROKER HOST:PORT} --describe --topic {TOPIC NAME}”
• “delivery.timeout.ms” – An upper bound on the time to report success or failure after a call to send()
returns.
• The default value is 120000 ms (2 minutes).
• If ”delivery.timeout.ms” is set to a very low value, it can cause batches to be expired too early.
• “batch.size” – The maximum size of a record batch.
• The default value is 16384 bytes (16 kB).
• If the message size is large, this configuration may need to be increased to allow more records per
batch. More records per batch means higher throughput and lower latency per record.
Batch expires before sending
13
• How to investigate this issue (cont’d):
• First, we need to identify whether this is caused by slow transfer time or slow processing.
• To check if it is slow transfer time, execute “ping {BROKER HOST}” from the producer host. The round trip time
(RTT) should be reasonable. For example: If both producer and Kafka brokers are in the same data center, the
RTT should be less than 10 ms, mostly should be under 1 ms.
• If ”ping” result is good (i.e. consistently under 10 ms with 0% packet loss), then network latency is unlikely
to be the cause.
• To check if it is slow processing, check the following on Kafka brokers:
• Number of connections on the Kafka broker with “netstat -n | grep 9092 | wc -l”. More than 1000
connections is usually too high and can cause slow processing or connectivity issue.
• Number of topic partitions per broker. More than 1500 partitions per broker is usually too high and can
cause slow processing. Check it with “kafka-topics --describe | awk ‘{print $5, $6}’ | sort | uniq –c”.
• If Kafka broker host has enough CPU and memory, then you can increment “num.replica.fetchers” to 2 or 3 to allow
more partitions per broker.
• Inter-broker ”ping” latency. If the brokers are running on multiple data center (e.g. multiple Availability
Zone), then this may be significant contributor to produce latency.
• CPU usage of Kafka brokers. Following JMX metrics also show the internal thread idle-ness if you need:
• kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent – if this is low (< 0.5), that
means it needs higher “num.io.threads”, if CPU allows.
• kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent – if this is low (< 0.5), that means it
needs higher “num.network.threads”, if CPU allows.
Not enough replicas error
14
• This means the number of replicas in ISR is less than “min.insync.replicas” configuration.
• The error looks like this:
• [2021-08-03 01:34:05,077] WARN [Producer clientId=producer-1] Got error produce response with
correlation id 3 on topic-partition test2-0, retrying (2147483646 attempts left). Error:
NOT_ENOUGH_REPLICAS (org.apache.kafka.clients.producer.internals.Sender)
• This error occurs when:
• Topic replication factor is 3.
• Topic configuration includes “min.insync.replicas=2”.
• Producer uses “acks=all” configuration.
Not enough replica error
15
• What is ISR? Short for “In Sync Replicas”. This means the follower replicas that are in sync
with the leader. In other word, the follower replicas that have all records that the leader
replica has.
• How can a replica become out of sync? Either because the broker is offline or replication
failure or slow replication.
• How to fix this error:
• If it is out of sync because Kafka broker being offline, start the broker hosting the offline replicas.
• If it is out of sync because of replication failure, fix the failure. This is separate discussion. But the most
common one is disk failure. If the disk storing the replica data is full, Kafka broker will stop replicating all
replicas on that disk.
• If it is out of sync because of slow replication, fix the slow replication. This is also separate discussion.
But the most common cause is inter-broker latency or too many topic partitions per broker.
Thank you. Any questions?
Ad

More Related Content

What's hot (20)

Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
Jiangjie Qin
 
Apache Kafka Introduction
Apache Kafka IntroductionApache Kafka Introduction
Apache Kafka Introduction
Amita Mirajkar
 
Kafka 101 and Developer Best Practices
Kafka 101 and Developer Best PracticesKafka 101 and Developer Best Practices
Kafka 101 and Developer Best Practices
confluent
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
Flink Forward
 
Kafka presentation
Kafka presentationKafka presentation
Kafka presentation
Mohammed Fazuluddin
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using Kafka
Knoldus Inc.
 
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 Forward
 
Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explained
confluent
 
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
HostedbyConfluent
 
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
Flink Forward
 
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
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
Jeff Holoman
 
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
confluent
 
APACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka StreamsAPACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka Streams
Ketan Gote
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafka
Jiangjie Qin
 
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 KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
HostedbyConfluent
 
An Introduction to Apache Kafka
An Introduction to Apache KafkaAn Introduction to Apache Kafka
An Introduction to Apache Kafka
Amir Sedighi
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
AIMDek Technologies
 
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
confluent
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
Jiangjie Qin
 
Apache Kafka Introduction
Apache Kafka IntroductionApache Kafka Introduction
Apache Kafka Introduction
Amita Mirajkar
 
Kafka 101 and Developer Best Practices
Kafka 101 and Developer Best PracticesKafka 101 and Developer Best Practices
Kafka 101 and Developer Best Practices
confluent
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
Flink Forward
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using Kafka
Knoldus Inc.
 
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 Forward
 
Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explained
confluent
 
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
HostedbyConfluent
 
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
Flink Forward
 
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
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
Jeff Holoman
 
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
Everything You Always Wanted to Know About Kafka’s Rebalance Protocol but Wer...
confluent
 
APACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka StreamsAPACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka Streams
Ketan Gote
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafka
Jiangjie Qin
 
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 KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
Introducing KRaft: Kafka Without Zookeeper With Colin McCabe | Current 2022
HostedbyConfluent
 
An Introduction to Apache Kafka
An Introduction to Apache KafkaAn Introduction to Apache Kafka
An Introduction to Apache Kafka
Amir Sedighi
 
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
confluent
 

Similar to Common issues with Apache Kafka® Producer (20)

Reliability Guarantees for Apache Kafka
Reliability Guarantees for Apache KafkaReliability Guarantees for Apache Kafka
Reliability Guarantees for Apache Kafka
confluent
 
Kafka reliability velocity 17
Kafka reliability   velocity 17Kafka reliability   velocity 17
Kafka reliability velocity 17
Gwen (Chen) Shapira
 
Apache Kafka Reliability
Apache Kafka Reliability Apache Kafka Reliability
Apache Kafka Reliability
Jeff Holoman
 
Developing Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache KafkaDeveloping Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache Kafka
Joe Stein
 
Kafka Explainaton
Kafka ExplainatonKafka Explainaton
Kafka Explainaton
NguyenChiHoangMinh
 
Exactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache KafkaExactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache Kafka
confluent
 
Kafka zero to hero
Kafka zero to heroKafka zero to hero
Kafka zero to hero
Avi Levi
 
Apache Kafka - From zero to hero
Apache Kafka - From zero to heroApache Kafka - From zero to hero
Apache Kafka - From zero to hero
Apache Kafka TLV
 
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
Levon Avakyan
 
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
 
Removing performance bottlenecks with Kafka Monitoring and topic configuration
Removing performance bottlenecks with Kafka Monitoring and topic configurationRemoving performance bottlenecks with Kafka Monitoring and topic configuration
Removing performance bottlenecks with Kafka Monitoring and topic configuration
Knoldus Inc.
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
Joe Stein
 
Envoy and Kafka
Envoy and KafkaEnvoy and Kafka
Envoy and Kafka
Adam Kotwasinski
 
Stream Processing with Apache Kafka and .NET
Stream Processing with Apache Kafka and .NETStream Processing with Apache Kafka and .NET
Stream Processing with Apache Kafka and .NET
confluent
 
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Otávio Carvalho
 
Apache kafka
Apache kafkaApache kafka
Apache kafka
Kumar Shivam
 
Understanding Apache Kafka P99 Latency at Scale
Understanding Apache Kafka P99 Latency at ScaleUnderstanding Apache Kafka P99 Latency at Scale
Understanding Apache Kafka P99 Latency at Scale
ScyllaDB
 
Kafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced ProducersKafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced Producers
Jean-Paul Azar
 
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE PlatformsFIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE
 
Load Balancing 101
Load Balancing 101Load Balancing 101
Load Balancing 101
HungWei Chiu
 
Reliability Guarantees for Apache Kafka
Reliability Guarantees for Apache KafkaReliability Guarantees for Apache Kafka
Reliability Guarantees for Apache Kafka
confluent
 
Apache Kafka Reliability
Apache Kafka Reliability Apache Kafka Reliability
Apache Kafka Reliability
Jeff Holoman
 
Developing Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache KafkaDeveloping Realtime Data Pipelines With Apache Kafka
Developing Realtime Data Pipelines With Apache Kafka
Joe Stein
 
Exactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache KafkaExactly-once Semantics in Apache Kafka
Exactly-once Semantics in Apache Kafka
confluent
 
Kafka zero to hero
Kafka zero to heroKafka zero to hero
Kafka zero to hero
Avi Levi
 
Apache Kafka - From zero to hero
Apache Kafka - From zero to heroApache Kafka - From zero to hero
Apache Kafka - From zero to hero
Apache Kafka TLV
 
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
Levon Avakyan
 
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
 
Removing performance bottlenecks with Kafka Monitoring and topic configuration
Removing performance bottlenecks with Kafka Monitoring and topic configurationRemoving performance bottlenecks with Kafka Monitoring and topic configuration
Removing performance bottlenecks with Kafka Monitoring and topic configuration
Knoldus Inc.
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
Joe Stein
 
Stream Processing with Apache Kafka and .NET
Stream Processing with Apache Kafka and .NETStream Processing with Apache Kafka and .NET
Stream Processing with Apache Kafka and .NET
confluent
 
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018Non-Kafkaesque Apache Kafka - Yottabyte 2018
Non-Kafkaesque Apache Kafka - Yottabyte 2018
Otávio Carvalho
 
Understanding Apache Kafka P99 Latency at Scale
Understanding Apache Kafka P99 Latency at ScaleUnderstanding Apache Kafka P99 Latency at Scale
Understanding Apache Kafka P99 Latency at Scale
ScyllaDB
 
Kafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced ProducersKafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced Producers
Jean-Paul Azar
 
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE PlatformsFIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE Tech Summit - Docker Swarm Secrets for Creating Great FIWARE Platforms
FIWARE
 
Load Balancing 101
Load Balancing 101Load Balancing 101
Load Balancing 101
HungWei Chiu
 
Ad

More from confluent (20)

Webinar Think Right - Shift Left - 19-03-2025.pptx
Webinar Think Right - Shift Left - 19-03-2025.pptxWebinar Think Right - Shift Left - 19-03-2025.pptx
Webinar Think Right - Shift Left - 19-03-2025.pptx
confluent
 
Migration, backup and restore made easy using Kannika
Migration, backup and restore made easy using KannikaMigration, backup and restore made easy using Kannika
Migration, backup and restore made easy using Kannika
confluent
 
Five Things You Need to Know About Data Streaming in 2025
Five Things You Need to Know About Data Streaming in 2025Five Things You Need to Know About Data Streaming in 2025
Five Things You Need to Know About Data Streaming in 2025
confluent
 
Data in Motion Tour Seoul 2024 - Keynote
Data in Motion Tour Seoul 2024 - KeynoteData in Motion Tour Seoul 2024 - Keynote
Data in Motion Tour Seoul 2024 - Keynote
confluent
 
Data in Motion Tour Seoul 2024 - Roadmap Demo
Data in Motion Tour Seoul 2024  - Roadmap DemoData in Motion Tour Seoul 2024  - Roadmap Demo
Data in Motion Tour Seoul 2024 - Roadmap Demo
confluent
 
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
confluent
 
Confluent per il settore FSI: Accelerare l'Innovazione con il Data Streaming...
Confluent per il settore FSI:  Accelerare l'Innovazione con il Data Streaming...Confluent per il settore FSI:  Accelerare l'Innovazione con il Data Streaming...
Confluent per il settore FSI: Accelerare l'Innovazione con il Data Streaming...
confluent
 
Data in Motion Tour 2024 Riyadh, Saudi Arabia
Data in Motion Tour 2024 Riyadh, Saudi ArabiaData in Motion Tour 2024 Riyadh, Saudi Arabia
Data in Motion Tour 2024 Riyadh, Saudi Arabia
confluent
 
Build a Real-Time Decision Support Application for Financial Market Traders w...
Build a Real-Time Decision Support Application for Financial Market Traders w...Build a Real-Time Decision Support Application for Financial Market Traders w...
Build a Real-Time Decision Support Application for Financial Market Traders w...
confluent
 
Strumenti e Strategie di Stream Governance con Confluent Platform
Strumenti e Strategie di Stream Governance con Confluent PlatformStrumenti e Strategie di Stream Governance con Confluent Platform
Strumenti e Strategie di Stream Governance con Confluent Platform
confluent
 
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not WeeksCompose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
confluent
 
Building Real-Time Gen AI Applications with SingleStore and Confluent
Building Real-Time Gen AI Applications with SingleStore and ConfluentBuilding Real-Time Gen AI Applications with SingleStore and Confluent
Building Real-Time Gen AI Applications with SingleStore and Confluent
confluent
 
Unlocking value with event-driven architecture by Confluent
Unlocking value with event-driven architecture by ConfluentUnlocking value with event-driven architecture by Confluent
Unlocking value with event-driven architecture by Confluent
confluent
 
Il Data Streaming per un’AI real-time di nuova generazione
Il Data Streaming per un’AI real-time di nuova generazioneIl Data Streaming per un’AI real-time di nuova generazione
Il Data Streaming per un’AI real-time di nuova generazione
confluent
 
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
confluent
 
Break data silos with real-time connectivity using Confluent Cloud Connectors
Break data silos with real-time connectivity using Confluent Cloud ConnectorsBreak data silos with real-time connectivity using Confluent Cloud Connectors
Break data silos with real-time connectivity using Confluent Cloud Connectors
confluent
 
Building API data products on top of your real-time data infrastructure
Building API data products on top of your real-time data infrastructureBuilding API data products on top of your real-time data infrastructure
Building API data products on top of your real-time data infrastructure
confluent
 
Speed Wins: From Kafka to APIs in Minutes
Speed Wins: From Kafka to APIs in MinutesSpeed Wins: From Kafka to APIs in Minutes
Speed Wins: From Kafka to APIs in Minutes
confluent
 
Evolving Data Governance for the Real-time Streaming and AI Era
Evolving Data Governance for the Real-time Streaming and AI EraEvolving Data Governance for the Real-time Streaming and AI Era
Evolving Data Governance for the Real-time Streaming and AI Era
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
 
Webinar Think Right - Shift Left - 19-03-2025.pptx
Webinar Think Right - Shift Left - 19-03-2025.pptxWebinar Think Right - Shift Left - 19-03-2025.pptx
Webinar Think Right - Shift Left - 19-03-2025.pptx
confluent
 
Migration, backup and restore made easy using Kannika
Migration, backup and restore made easy using KannikaMigration, backup and restore made easy using Kannika
Migration, backup and restore made easy using Kannika
confluent
 
Five Things You Need to Know About Data Streaming in 2025
Five Things You Need to Know About Data Streaming in 2025Five Things You Need to Know About Data Streaming in 2025
Five Things You Need to Know About Data Streaming in 2025
confluent
 
Data in Motion Tour Seoul 2024 - Keynote
Data in Motion Tour Seoul 2024 - KeynoteData in Motion Tour Seoul 2024 - Keynote
Data in Motion Tour Seoul 2024 - Keynote
confluent
 
Data in Motion Tour Seoul 2024 - Roadmap Demo
Data in Motion Tour Seoul 2024  - Roadmap DemoData in Motion Tour Seoul 2024  - Roadmap Demo
Data in Motion Tour Seoul 2024 - Roadmap Demo
confluent
 
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
From Stream to Screen: Real-Time Data Streaming to Web Frontends with Conflue...
confluent
 
Confluent per il settore FSI: Accelerare l'Innovazione con il Data Streaming...
Confluent per il settore FSI:  Accelerare l'Innovazione con il Data Streaming...Confluent per il settore FSI:  Accelerare l'Innovazione con il Data Streaming...
Confluent per il settore FSI: Accelerare l'Innovazione con il Data Streaming...
confluent
 
Data in Motion Tour 2024 Riyadh, Saudi Arabia
Data in Motion Tour 2024 Riyadh, Saudi ArabiaData in Motion Tour 2024 Riyadh, Saudi Arabia
Data in Motion Tour 2024 Riyadh, Saudi Arabia
confluent
 
Build a Real-Time Decision Support Application for Financial Market Traders w...
Build a Real-Time Decision Support Application for Financial Market Traders w...Build a Real-Time Decision Support Application for Financial Market Traders w...
Build a Real-Time Decision Support Application for Financial Market Traders w...
confluent
 
Strumenti e Strategie di Stream Governance con Confluent Platform
Strumenti e Strategie di Stream Governance con Confluent PlatformStrumenti e Strategie di Stream Governance con Confluent Platform
Strumenti e Strategie di Stream Governance con Confluent Platform
confluent
 
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not WeeksCompose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
Compose Gen-AI Apps With Real-Time Data - In Minutes, Not Weeks
confluent
 
Building Real-Time Gen AI Applications with SingleStore and Confluent
Building Real-Time Gen AI Applications with SingleStore and ConfluentBuilding Real-Time Gen AI Applications with SingleStore and Confluent
Building Real-Time Gen AI Applications with SingleStore and Confluent
confluent
 
Unlocking value with event-driven architecture by Confluent
Unlocking value with event-driven architecture by ConfluentUnlocking value with event-driven architecture by Confluent
Unlocking value with event-driven architecture by Confluent
confluent
 
Il Data Streaming per un’AI real-time di nuova generazione
Il Data Streaming per un’AI real-time di nuova generazioneIl Data Streaming per un’AI real-time di nuova generazione
Il Data Streaming per un’AI real-time di nuova generazione
confluent
 
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
Unleashing the Future: Building a Scalable and Up-to-Date GenAI Chatbot with ...
confluent
 
Break data silos with real-time connectivity using Confluent Cloud Connectors
Break data silos with real-time connectivity using Confluent Cloud ConnectorsBreak data silos with real-time connectivity using Confluent Cloud Connectors
Break data silos with real-time connectivity using Confluent Cloud Connectors
confluent
 
Building API data products on top of your real-time data infrastructure
Building API data products on top of your real-time data infrastructureBuilding API data products on top of your real-time data infrastructure
Building API data products on top of your real-time data infrastructure
confluent
 
Speed Wins: From Kafka to APIs in Minutes
Speed Wins: From Kafka to APIs in MinutesSpeed Wins: From Kafka to APIs in Minutes
Speed Wins: From Kafka to APIs in Minutes
confluent
 
Evolving Data Governance for the Real-time Streaming and AI Era
Evolving Data Governance for the Real-time Streaming and AI EraEvolving Data Governance for the Real-time Streaming and AI Era
Evolving Data Governance for the Real-time Streaming and AI Era
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
 
Ad

Recently uploaded (20)

Buckeye Dreamin' 2023: De-fogging Debug Logs
Buckeye Dreamin' 2023: De-fogging Debug LogsBuckeye Dreamin' 2023: De-fogging Debug Logs
Buckeye Dreamin' 2023: De-fogging Debug Logs
Lynda Kane
 
Datastucture-Unit 4-Linked List Presentation.pptx
Datastucture-Unit 4-Linked List Presentation.pptxDatastucture-Unit 4-Linked List Presentation.pptx
Datastucture-Unit 4-Linked List Presentation.pptx
kaleeswaric3
 
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
Lynda Kane
 
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager APIUiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPathCommunity
 
Network Security. Different aspects of Network Security.
Network Security. Different aspects of Network Security.Network Security. Different aspects of Network Security.
Network Security. Different aspects of Network Security.
gregtap1
 
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5..."Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
Fwdays
 
AI and Data Privacy in 2025: Global Trends
AI and Data Privacy in 2025: Global TrendsAI and Data Privacy in 2025: Global Trends
AI and Data Privacy in 2025: Global Trends
InData Labs
 
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc
 
Into The Box Conference Keynote Day 1 (ITB2025)
Into The Box Conference Keynote Day 1 (ITB2025)Into The Box Conference Keynote Day 1 (ITB2025)
Into The Box Conference Keynote Day 1 (ITB2025)
Ortus Solutions, Corp
 
How Can I use the AI Hype in my Business Context?
How Can I use the AI Hype in my Business Context?How Can I use the AI Hype in my Business Context?
How Can I use the AI Hype in my Business Context?
Daniel Lehner
 
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
Alan Dix
 
Electronic_Mail_Attacks-1-35.pdf by xploit
Electronic_Mail_Attacks-1-35.pdf by xploitElectronic_Mail_Attacks-1-35.pdf by xploit
Electronic_Mail_Attacks-1-35.pdf by xploit
niftliyevhuseyn
 
Learn the Basics of Agile Development: Your Step-by-Step Guide
Learn the Basics of Agile Development: Your Step-by-Step GuideLearn the Basics of Agile Development: Your Step-by-Step Guide
Learn the Basics of Agile Development: Your Step-by-Step Guide
Marcel David
 
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
SOFTTECHHUB
 
Role of Data Annotation Services in AI-Powered Manufacturing
Role of Data Annotation Services in AI-Powered ManufacturingRole of Data Annotation Services in AI-Powered Manufacturing
Role of Data Annotation Services in AI-Powered Manufacturing
Andrew Leo
 
Dev Dives: Automate and orchestrate your processes with UiPath Maestro
Dev Dives: Automate and orchestrate your processes with UiPath MaestroDev Dives: Automate and orchestrate your processes with UiPath Maestro
Dev Dives: Automate and orchestrate your processes with UiPath Maestro
UiPathCommunity
 
Technology Trends in 2025: AI and Big Data Analytics
Technology Trends in 2025: AI and Big Data AnalyticsTechnology Trends in 2025: AI and Big Data Analytics
Technology Trends in 2025: AI and Big Data Analytics
InData Labs
 
Splunk Security Update | Public Sector Summit Germany 2025
Splunk Security Update | Public Sector Summit Germany 2025Splunk Security Update | Public Sector Summit Germany 2025
Splunk Security Update | Public Sector Summit Germany 2025
Splunk
 
What is Model Context Protocol(MCP) - The new technology for communication bw...
What is Model Context Protocol(MCP) - The new technology for communication bw...What is Model Context Protocol(MCP) - The new technology for communication bw...
What is Model Context Protocol(MCP) - The new technology for communication bw...
Vishnu Singh Chundawat
 
Automation Hour 1/28/2022: Capture User Feedback from Anywhere
Automation Hour 1/28/2022: Capture User Feedback from AnywhereAutomation Hour 1/28/2022: Capture User Feedback from Anywhere
Automation Hour 1/28/2022: Capture User Feedback from Anywhere
Lynda Kane
 
Buckeye Dreamin' 2023: De-fogging Debug Logs
Buckeye Dreamin' 2023: De-fogging Debug LogsBuckeye Dreamin' 2023: De-fogging Debug Logs
Buckeye Dreamin' 2023: De-fogging Debug Logs
Lynda Kane
 
Datastucture-Unit 4-Linked List Presentation.pptx
Datastucture-Unit 4-Linked List Presentation.pptxDatastucture-Unit 4-Linked List Presentation.pptx
Datastucture-Unit 4-Linked List Presentation.pptx
kaleeswaric3
 
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
#AdminHour presents: Hour of Code2018 slide deck from 12/6/2018
Lynda Kane
 
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager APIUiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPath Community Berlin: Orchestrator API, Swagger, and Test Manager API
UiPathCommunity
 
Network Security. Different aspects of Network Security.
Network Security. Different aspects of Network Security.Network Security. Different aspects of Network Security.
Network Security. Different aspects of Network Security.
gregtap1
 
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5..."Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
"Client Partnership — the Path to Exponential Growth for Companies Sized 50-5...
Fwdays
 
AI and Data Privacy in 2025: Global Trends
AI and Data Privacy in 2025: Global TrendsAI and Data Privacy in 2025: Global Trends
AI and Data Privacy in 2025: Global Trends
InData Labs
 
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc Webinar: Consumer Expectations vs Corporate Realities on Data Broker...
TrustArc
 
Into The Box Conference Keynote Day 1 (ITB2025)
Into The Box Conference Keynote Day 1 (ITB2025)Into The Box Conference Keynote Day 1 (ITB2025)
Into The Box Conference Keynote Day 1 (ITB2025)
Ortus Solutions, Corp
 
How Can I use the AI Hype in my Business Context?
How Can I use the AI Hype in my Business Context?How Can I use the AI Hype in my Business Context?
How Can I use the AI Hype in my Business Context?
Daniel Lehner
 
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
AI Changes Everything – Talk at Cardiff Metropolitan University, 29th April 2...
Alan Dix
 
Electronic_Mail_Attacks-1-35.pdf by xploit
Electronic_Mail_Attacks-1-35.pdf by xploitElectronic_Mail_Attacks-1-35.pdf by xploit
Electronic_Mail_Attacks-1-35.pdf by xploit
niftliyevhuseyn
 
Learn the Basics of Agile Development: Your Step-by-Step Guide
Learn the Basics of Agile Development: Your Step-by-Step GuideLearn the Basics of Agile Development: Your Step-by-Step Guide
Learn the Basics of Agile Development: Your Step-by-Step Guide
Marcel David
 
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
AI EngineHost Review: Revolutionary USA Datacenter-Based Hosting with NVIDIA ...
SOFTTECHHUB
 
Role of Data Annotation Services in AI-Powered Manufacturing
Role of Data Annotation Services in AI-Powered ManufacturingRole of Data Annotation Services in AI-Powered Manufacturing
Role of Data Annotation Services in AI-Powered Manufacturing
Andrew Leo
 
Dev Dives: Automate and orchestrate your processes with UiPath Maestro
Dev Dives: Automate and orchestrate your processes with UiPath MaestroDev Dives: Automate and orchestrate your processes with UiPath Maestro
Dev Dives: Automate and orchestrate your processes with UiPath Maestro
UiPathCommunity
 
Technology Trends in 2025: AI and Big Data Analytics
Technology Trends in 2025: AI and Big Data AnalyticsTechnology Trends in 2025: AI and Big Data Analytics
Technology Trends in 2025: AI and Big Data Analytics
InData Labs
 
Splunk Security Update | Public Sector Summit Germany 2025
Splunk Security Update | Public Sector Summit Germany 2025Splunk Security Update | Public Sector Summit Germany 2025
Splunk Security Update | Public Sector Summit Germany 2025
Splunk
 
What is Model Context Protocol(MCP) - The new technology for communication bw...
What is Model Context Protocol(MCP) - The new technology for communication bw...What is Model Context Protocol(MCP) - The new technology for communication bw...
What is Model Context Protocol(MCP) - The new technology for communication bw...
Vishnu Singh Chundawat
 
Automation Hour 1/28/2022: Capture User Feedback from Anywhere
Automation Hour 1/28/2022: Capture User Feedback from AnywhereAutomation Hour 1/28/2022: Capture User Feedback from Anywhere
Automation Hour 1/28/2022: Capture User Feedback from Anywhere
Lynda Kane
 

Common issues with Apache Kafka® Producer

  • 1. Common issues with Apache Kafka® Producer Badai Aqrandista, Senior Technical Support Engineer
  • 2. Introduction 2 • My name is BADAI AQRANDISTA • I started as a web developer, building website with Perl and PHP in 2005. • Experience supporting applications on Linux/UNIX environment, from hotel booking engine, telecommunication billing system, and mining equipment monitoring system. • Currently working for Confluent as Senior Technical Support Engineer.
  • 3. Kafka in a nutshell 3 • Kafka is a Pub/Sub system • Kafka Producer sends record into Kafka broker • Kafka Consumer fetches record from Kafka broker • Kafka broker persists any data it receives until retention period expires PRODUCER CONSUME R
  • 5. Kafka Producer Internals 5 • KafkaProducer API: • public Future<RecordMetadata> send(ProducerRecord<K,V> record) • public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback callback) • KafkaProducer#send method is asynchronous. • It does not immediately send the record to Kafka broker. • It puts the record in an internal queue and an internal queue will send multiple records as a batch. Batch Record Key Value Record Key Value Record Key Value
  • 6. Kafka Producer Internals 6 • Each Kafka Producer batch corresponds to a partitions. • Kafka Producer determines the batch to append a record to based on the record key. • If record key is “null”, Kafka Producer will choose the batch randomly. • If record key is not “null”, Kafka Producer will use the hash of the record key to determine the partition number. • One or more batches are sent to the Kafka broker in a PRODUCE request.
  • 7. Kafka Producer Internals 7 • Kafka Producer internal thread sends a batch to Kafka broker based on these configuration: • “batch.size” – defaults to 16 kB • “linger.ms” – defaults to 0 • So, Kafka Producer internal thread sends a batch to Kafka broker when: • The total size of records in the batch exceeds “batch.size”, or • The time since batch creation exceeds “linger.ms”, or • Kafka Producer ”flush()” method is called (directly or indirectly via “close()”). • Kafka Producer only creates one connection to each broker. • In the end, every batch for a Kafka broker must be sent sequentially through this one connection. • The maximum number of batches sent to each broker at any one time is controlled by “max.in.flight.requests.per.connection”, which defaults to 5.
  • 9. Kafka Producer Issues 9 1. Failure to connect to Kafka broker 2. Record is too large 3. Batch expires before sending 4. Not enough replicas error
  • 10. Failure to connect to Kafka broker 10 • This error is not obvious, but it means failure to connect to Kafka broker. • The error message looks like this: • [2021-08-02 12:57:44,097] WARN [Producer clientId=producer-1] Connection to node -1 (kafka1/172.20.0.6:9093) could not be established. Broker may not be available. (org.apache.kafka.clients.NetworkClient) • How to fix this: • Check the broker configuration to confirm the listener port and security protocol • Check the hostname or the IP address of the broker • Confirm that Kafka Producer’s bootstrap.server configuration is correct • Confirm that connectivity exists between Kafka Producer’s host and Kafka broker hosts with commands such as: • ping {BROKER_HOST} • nc {BROKER_HOST} {BROKER PORT} • openssl s_client -connect {BROKER_HOST}:{BROKER_PORT}
  • 11. Record is too large 11 • This error is because the record size is greater than “max.request.size” configuration, which defaults to 1048576 (1 MB). • The error message is like this: • org.apache.kafka.common.errors.RecordTooLargeException: The message is 1600088 bytes when serialized which is larger than 1048576, which is the value of the max.request.size configuration. • How to fix it: • Reduce the record size. This requires a change in the application that generates the record. • If you cannot reduce the record size, you can increase producer configuration “max.request.size”. If you do this, you also need to increase topic configuration “max.message.bytes”. • Note: “max.request.size” is the maximum request size AFTER serialization but BEFORE compression. So, setting compression will not fix this.
  • 12. Batch expires before sending 12 • This error is a symptom of slow transfer time (on network) or slow processing (on Kafka broker). • The error looks like this: • org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for test1-0:1500 ms has passed since batch creation • Sanity checks: • Is the topic partition online? Topic partition is online if one or more Kafka brokers hosting the replicas are online. • Use “kafka-topics --bootstrap-server {BROKER HOST:PORT} --describe --topic {TOPIC NAME}” • “delivery.timeout.ms” – An upper bound on the time to report success or failure after a call to send() returns. • The default value is 120000 ms (2 minutes). • If ”delivery.timeout.ms” is set to a very low value, it can cause batches to be expired too early. • “batch.size” – The maximum size of a record batch. • The default value is 16384 bytes (16 kB). • If the message size is large, this configuration may need to be increased to allow more records per batch. More records per batch means higher throughput and lower latency per record.
  • 13. Batch expires before sending 13 • How to investigate this issue (cont’d): • First, we need to identify whether this is caused by slow transfer time or slow processing. • To check if it is slow transfer time, execute “ping {BROKER HOST}” from the producer host. The round trip time (RTT) should be reasonable. For example: If both producer and Kafka brokers are in the same data center, the RTT should be less than 10 ms, mostly should be under 1 ms. • If ”ping” result is good (i.e. consistently under 10 ms with 0% packet loss), then network latency is unlikely to be the cause. • To check if it is slow processing, check the following on Kafka brokers: • Number of connections on the Kafka broker with “netstat -n | grep 9092 | wc -l”. More than 1000 connections is usually too high and can cause slow processing or connectivity issue. • Number of topic partitions per broker. More than 1500 partitions per broker is usually too high and can cause slow processing. Check it with “kafka-topics --describe | awk ‘{print $5, $6}’ | sort | uniq –c”. • If Kafka broker host has enough CPU and memory, then you can increment “num.replica.fetchers” to 2 or 3 to allow more partitions per broker. • Inter-broker ”ping” latency. If the brokers are running on multiple data center (e.g. multiple Availability Zone), then this may be significant contributor to produce latency. • CPU usage of Kafka brokers. Following JMX metrics also show the internal thread idle-ness if you need: • kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent – if this is low (< 0.5), that means it needs higher “num.io.threads”, if CPU allows. • kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent – if this is low (< 0.5), that means it needs higher “num.network.threads”, if CPU allows.
  • 14. Not enough replicas error 14 • This means the number of replicas in ISR is less than “min.insync.replicas” configuration. • The error looks like this: • [2021-08-03 01:34:05,077] WARN [Producer clientId=producer-1] Got error produce response with correlation id 3 on topic-partition test2-0, retrying (2147483646 attempts left). Error: NOT_ENOUGH_REPLICAS (org.apache.kafka.clients.producer.internals.Sender) • This error occurs when: • Topic replication factor is 3. • Topic configuration includes “min.insync.replicas=2”. • Producer uses “acks=all” configuration.
  • 15. Not enough replica error 15 • What is ISR? Short for “In Sync Replicas”. This means the follower replicas that are in sync with the leader. In other word, the follower replicas that have all records that the leader replica has. • How can a replica become out of sync? Either because the broker is offline or replication failure or slow replication. • How to fix this error: • If it is out of sync because Kafka broker being offline, start the broker hosting the offline replicas. • If it is out of sync because of replication failure, fix the failure. This is separate discussion. But the most common one is disk failure. If the disk storing the replica data is full, Kafka broker will stop replicating all replicas on that disk. • If it is out of sync because of slow replication, fix the slow replication. This is also separate discussion. But the most common cause is inter-broker latency or too many topic partitions per broker.
  • 16. Thank you. Any questions?