SlideShare a Scribd company logo
1 of 35
Apache Kafka Best
Practices
Manikumar Reddy
@omkreddy
2 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Apache Kafka
 Core APIs
– The Producer API
– The Consumer API
– The Connector API
– The Streams API
 Broad classes of applications
– Building real-time streaming data pipelines
– Building real-time streaming applications
– core building block in other data systems
3 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Key Concepts and Terminology
4 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Component Layout
5 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Hardware Guidance
Cluster Size Memory CPU Storage
Kafka Brokers 3+
24G+ (for small)
64GB+ (for large)
Multi- core
processors( 12 CPU+
core), Hyper
threading enabled
6+ x 1TB dedicated
disks( RAID or JBOD)
Zookeeper
3 (for small)
5 (for large)
8GB+ (for small)
24GB+ (for large)
2 core +
SSD for Transaction
logs
6 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
OS Tuning
 OS Page Cache
– Ex: Allocate to hold all the active segments of the log.
 File descriptor limits : >100k
 less swapping
 Tcp tuning
 JVM Configs
– Java 8 with G1 Collector
– 6-8 GB heap
7 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Disk Storage
 Use multiple disk spindles, dedicated to kafka
 JBOD vs RAID10
 JBOD
– Gives all the disk I/O
 JBOD Limitations
– any disk failure causes an unclean shutdown and requires lengthy recovery
– data is not distributed consistently across disks
– Multiple directories
 KIP-112/113
– necessary tools for users to manage JBOD
– Intelligent partition assignment
– On disk failure, broker can serve replicas on the good disks
– re-assign replicas between disks of the same broker
8 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
RAID
 RAID10
– Can survive single disk failure
– Performance and protection
– balance load across disks
– Single mount point
– Performance hit and reduces the space
 File System
– EXT or XFS
– SSD
– Issues on NFS.
– SAN, NAS
9 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Basic Monitoring
 CPU Load
 Network Metrics
 File Handle Usage
 Disk Space
 Disk I/O Performance
 Garbage Collection
 ZooKeeper Monitoring
10 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Replication
 Partition has replicas – Leader replica, Follower replicas
 Leader maintains in-sync-replicas (ISR)
– replica.lag.time.max.ms, num.replica.fetchers
– min.insync.replica – used by producer to ensure greater durability
https://www.slideshare.net/junrao/kafka-replication-apachecon2013
11 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Under Replicated Partitions
 Number of partitions which are not fully replicated within the cluster
 Mbean - kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions
 ISR Shrink/Expand Rate
 Under Replicated Partitions
– Lost Broker?
– Controller Issues
– Zookeeper Issues
– Network Issues
 Solutions
– Tune the ISR settings
– Expand brokers
12 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Controller
 Manages Partitions Life cycle
 Avoid controller's ZK session expires
– Soft failures – ISR Churn/Under replicated partitions
– ZK Server performance
– Long GC pauses on Broker
– Bad network configuration
 Monitoring
– Mbean : kafka.controller:type=KafkaController,name=ActiveControllerCount
– only one broker in the cluster should have 1
– LeaderElectionRate
13 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Unclean leader election
 Enable replicas not in the ISR set to be elected as leader
 Availability vs correctness
– By-default kafka chooses availability
 Monitoring
– Mbean : kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec
 Default will be changed in next release
14 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Broker Configs
 log.retention.{ms, minutes, hours} , log.retention.bytes
 message.max.bytes, replica.fetch.max.bytes
 delete.topic.enable
 unclean.leader.election.enable = false
 min.insync.replicas = 2
 replica.lag.time.max.ms, num.replica.fetchers
 replica.fetch.response.max.bytes
 zookeeper.session.timeout.ms = 30s
 num.io.threads
15 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Cluster Sizing
 Broker Sizing
– Partition count on each broker (<2K)
– Keep partition size on disk manageable (under 25GB per partition )
 Cluster Size (no. of brokers)
– how much retention we need
– how much traffic cluster is getting
 Cluster Expansion
– Disk usage on the log segments partition should stay under 60%
– Network usage on each broker should stay under 75%
 Cluster Monitoring
– Keep cluster balanced
– Ensure that partitions of a topic are fairly distributed across brokers
– Ensure that nodes in a cluster are not running out of disk and network
16 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Broker Monitoring
 Partition Counts
– Mbean: kafka.server:type=ReplicaManager,name=PartitionCount
 Leader replica counts
– Mbean: kafka.server:type=ReplicaManager,name=LeaderCount
 ISR Shrink Rate/ISR expansion rate
– kafka.server:type=ReplicaManager,name=IsrExpandsPerSec
 Message in rate/Byte in rate/Byte out rate
 NetworkProcessorAvgIdlePercent
 RequestHandlerAvgIdlePercent
17 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Topic Sizing
 No. of partitions
– Have at least as many partitions as there are consumers in the largest group
– topic is very busy – more partitions
– Keep partition size on disk manageable (under 25GB per partition )
– Take into account any other application requirements
– Special use cases – single partition
 Keyed messages
– enough partitions to deal with future growth
 expanding partitions
– whenever the size of the partition on disk is larger than threshold
18 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Choosing Partitions
 Based on throughput requirements one can pick a rough number of partitions.
– Lets call the throughput from producer to a single partition is P
– Throughput from a single partition to a consumer is C
– Target throughput is T
– At least max (T/P, T/C)
 More Partitions
– More open file handles
– May increase unavailability
– May increase end-to-end latency
– More memory for clients
19 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Quotas
 Protect from bad clients and maintain SLAs
 byte-rate thresholds on produce and fetch requests
 can be applied to (user, client-id), user or client-id groups.
 Server delays the responses
 Broker Metrics for monitoring – throttle-rate, byte-rate
 replica.fetch.response.max.bytes
– Limit memory usage of replica fetch response
 Limiting bandwidth usage during data migration
– kafka-reassign-partitions.sh -- -throttle option
20 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Producer
 User new java based clients
 Test in your Environment
– kafka-producer-perf-test.sh
 Memory
 CPU
 Batch Compression
 Avoid large messages
– creates more memory pressure
– slows down the brokers
21 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Critical Configs
 batch.size
– size based batching
– larger size -> high throughput, higher latency
 linger.ms
– time based batching
– larger size -> high throughput, higher latency
 max.in.flight.requests.per.connection
– Better throughput, affects ordering
 compression.type
– adding more user threads can help throughput
 acks
– Affects message durability
22 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Performance tuning
 If throughput < network capacity
– Add more user threads
– Increase batch size
– Add more producers instances
– Add more partitions
 Latency when acks = -1
– Increase num.replica.fetchers
 Cross datacenter data transfer
– Tune socket buffer settings, OS tcp buffer settings
23 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Producer Monitoring
 batch-size-avg
 compression-rate-avg
 waiting-threads
 buffer-available-bytes
 record-queue-time-max
 record-send-rate
 records-per-request-avg
24 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Consumer
 Test in your Environment
– kafka-consumer-perf-test.sh
 Throughput Issues
– not enough partitions
– OS Page Cache - allocate enough to hold all the messages for your consumers for say, 30s
– Application/Processing logic
 Offsets topic
– __consumer_offsets
– offsets.topic.replication.factor
– offsets.retention.minutes
– Monitor ISR, topic size
 Slow offset commits
– commit async, manual commits
25 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Consumer Configs
 fetch.min.bytes and fetch.max.wait.ms
 max.poll.interval.ms
 max.poll.records
 session.timeout.ms
 Consumer Rebalance
– check timeouts
– check processing times/logic
– GC Issues
 Tune network settings
26 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Consumer Monitoring
 Whether or not the consumer is keeping up with the messages that are being produced
 Consumer Lag: Difference between the end of the log and the consumer offset
 Monitoring
– Metrics Monitoring - records-lag-max
– bin/kafka-consumer-groups.sh
– LinkedIn’s Burrow for consumer monitoring
 Decreasing Lag
– Analyze consumer - GC Issues, hung instance
– Add more consumer Instances
– increase the number of partitions and consumers
27 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
No data loss settings
 Producer
– block.on.buffer.full=true
– retries=Long.MAX_VALUE
– acks=all
– max.in.flight.requests.per.connection=1
– close producer
 Broker
– replication factor >= 3
– min.insync.replicas=2
– disable unclean leader election
 Consumer
– disable auto.offset.commit
– Commit offsets only after the messages are processed
28 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Authorizer - Ranger Auditing
29 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Mirror Maker
 Tool to mirror a source Kafka cluster into a target (mirror) Kafka cluster
30 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Mirror Maker
 Run multiple mirroring processes
– high fault-tolerance
– high throughput
 --num.streams option to specify the number of consumer threads
– no.of threads in num.streams
31 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Kafka Mirror Maker
 Consumer and source cluster socket buffer sizes
– high value for the socket buffer size
– consumer's fetch size
– OS networking Tuning
 Source and Target Clusters are independent entities
– Can be different numbers of partitions
– offsets will not be the same.
– partitioning order is preserved on a per-key basis.
 Create topics in target cluster
 Monitor whether a mirror is keeping up
– Consumer Lag
 Running In Secure Clusters
– We recommend to use SSL
– We can run MM on source cluster
32 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Open source Operational Tools
 Ambari Metrics
– https://docs.hortonworks.com/HDPDocuments/Ambari-2.4.2.0/bk_ambari-user-
guide/content/grafana_kafka_dashboards.html
 Removing brokers and rebalancing partitions in a cluster
– https://github.com/linkedin/kafka-tools
 Consumer Lag Monitoring
– Burrow (https://github.com/linkedin/Burrow)
 Kafka Manager - https://github.com/yahoo/kafka-manager
33 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Apache Kafka 0.10.2 release
 Includes 15 KIPs, over 200 bug fixes and improvements
 The newest Java Clients now support older brokers (0.10.0 and higher)
 Separation of Internal and External traffic
 Create Topic Policy
 Security Improvements
– Support for SASL/SCRAM mechanisms
– Dynamic JAAS configuration for Kafka clients
– Support for authentication of multiple Kafka clients in single JVM
 Producer and Consumer Improvements
 Connect API & Streams API improvements
34 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
Thank You
35 © Hortonworks Inc. 2011 – 2017. All Rights Reserved
References
 http://kafka.apache.org/documentation.html
 https://community.hortonworks.com/articles/80813/kafka-best-practices-1.html
 https://www.slideshare.net/JiangjieQin/producer-performance-tuning-for-apache-
kafka-63147600
 https://www.slideshare.net/ToddPalino/tuning-kafka-for-fun-and-profit
 https://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-
49753844
 https://www.slideshare.net/ToddPalino/putting-kafka-into-overdrive
 https://www.confluent.io/blog/how-to-choose-the-number-of-topicspartitions-in-a-
kafka-cluster/

More Related Content

What's hot

Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안SANG WON PARK
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache KafkaJeff Holoman
 
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
 
Apache Kafka - Martin Podval
Apache Kafka - Martin PodvalApache Kafka - Martin Podval
Apache Kafka - Martin PodvalMartin Podval
 
Apache Kafka Fundamentals for Architects, Admins and Developers
Apache Kafka Fundamentals for Architects, Admins and DevelopersApache Kafka Fundamentals for Architects, Admins and Developers
Apache Kafka Fundamentals for Architects, Admins and Developersconfluent
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in KafkaJoel Koshy
 
Improving Kafka at-least-once performance at Uber
Improving Kafka at-least-once performance at UberImproving Kafka at-least-once performance at Uber
Improving Kafka at-least-once performance at UberYing Zheng
 
Kafka at Peak Performance
Kafka at Peak PerformanceKafka at Peak Performance
Kafka at Peak PerformanceTodd Palino
 
Getting Started with Confluent Schema Registry
Getting Started with Confluent Schema RegistryGetting Started with Confluent Schema Registry
Getting Started with Confluent Schema Registryconfluent
 
Disaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache KafkaDisaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache Kafkaconfluent
 
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
 
Kafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platformKafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platformJean-Paul Azar
 
Kafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced ProducersKafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced ProducersJean-Paul Azar
 
Securing Kafka
Securing Kafka Securing Kafka
Securing Kafka confluent
 
An Introduction to Apache Kafka
An Introduction to Apache KafkaAn Introduction to Apache Kafka
An Introduction to Apache KafkaAmir Sedighi
 

What's hot (20)

Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
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)
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
 
Apache Kafka - Martin Podval
Apache Kafka - Martin PodvalApache Kafka - Martin Podval
Apache Kafka - Martin Podval
 
kafka
kafkakafka
kafka
 
Apache Kafka Fundamentals for Architects, Admins and Developers
Apache Kafka Fundamentals for Architects, Admins and DevelopersApache Kafka Fundamentals for Architects, Admins and Developers
Apache Kafka Fundamentals for Architects, Admins and Developers
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in Kafka
 
Improving Kafka at-least-once performance at Uber
Improving Kafka at-least-once performance at UberImproving Kafka at-least-once performance at Uber
Improving Kafka at-least-once performance at Uber
 
Kafka at Peak Performance
Kafka at Peak PerformanceKafka at Peak Performance
Kafka at Peak Performance
 
Getting Started with Confluent Schema Registry
Getting Started with Confluent Schema RegistryGetting Started with Confluent Schema Registry
Getting Started with Confluent Schema Registry
 
Apache kafka
Apache kafkaApache kafka
Apache kafka
 
Disaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache KafkaDisaster Recovery Plans for Apache Kafka
Disaster Recovery Plans for Apache Kafka
 
Envoy and Kafka
Envoy and KafkaEnvoy and Kafka
Envoy and Kafka
 
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?
 
Kafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platformKafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platform
 
Kafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced ProducersKafka Tutorial: Advanced Producers
Kafka Tutorial: Advanced Producers
 
Securing Kafka
Securing Kafka Securing Kafka
Securing Kafka
 
An Introduction to Apache Kafka
An Introduction to Apache KafkaAn Introduction to Apache Kafka
An Introduction to Apache Kafka
 

Viewers also liked

Deep Dive into Apache Kafka
Deep Dive into Apache KafkaDeep Dive into Apache Kafka
Deep Dive into Apache Kafkaconfluent
 
Intro to Apache Kafka
Intro to Apache KafkaIntro to Apache Kafka
Intro to Apache KafkaJason Hubbard
 
MapR Streams and MapR Converged Data Platform
MapR Streams and MapR Converged Data PlatformMapR Streams and MapR Converged Data Platform
MapR Streams and MapR Converged Data PlatformMapR Technologies
 
Microservices in the Apache Kafka Ecosystem
Microservices in the Apache Kafka EcosystemMicroservices in the Apache Kafka Ecosystem
Microservices in the Apache Kafka Ecosystemconfluent
 
Self-Service Data Science for Leveraging ML & AI on All of Your Data
Self-Service Data Science for Leveraging ML & AI on All of Your DataSelf-Service Data Science for Leveraging ML & AI on All of Your Data
Self-Service Data Science for Leveraging ML & AI on All of Your DataMapR Technologies
 
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...Michael Noll
 
Processing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaProcessing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaMatthew Howlett
 
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017Michael Noll
 
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision Tree
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision TreeApache Kafka vs RabbitMQ: Fit For Purpose / Decision Tree
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision TreeSlim Baltagi
 
Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Michael Noll
 
Automatic Detection, Classification and Authorization of Sensitive Personal D...
Automatic Detection, Classification and Authorization of Sensitive Personal D...Automatic Detection, Classification and Authorization of Sensitive Personal D...
Automatic Detection, Classification and Authorization of Sensitive Personal D...DataWorks Summit/Hadoop Summit
 
Best Practices for Enterprise User Management in Hadoop Environment
Best Practices for Enterprise User Management in Hadoop EnvironmentBest Practices for Enterprise User Management in Hadoop Environment
Best Practices for Enterprise User Management in Hadoop EnvironmentDataWorks Summit/Hadoop Summit
 
MaaS (Model as a Service): Modern Streaming Data Science with Apache Metron
MaaS (Model as a Service): Modern Streaming Data Science with Apache MetronMaaS (Model as a Service): Modern Streaming Data Science with Apache Metron
MaaS (Model as a Service): Modern Streaming Data Science with Apache MetronDataWorks Summit
 
Tuning kafka pipelines
Tuning kafka pipelinesTuning kafka pipelines
Tuning kafka pipelinesSumant Tambe
 
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...DataWorks Summit/Hadoop Summit
 

Viewers also liked (20)

Deep Dive into Apache Kafka
Deep Dive into Apache KafkaDeep Dive into Apache Kafka
Deep Dive into Apache Kafka
 
Kafka internals
Kafka internalsKafka internals
Kafka internals
 
Intro to Apache Kafka
Intro to Apache KafkaIntro to Apache Kafka
Intro to Apache Kafka
 
MapR Streams and MapR Converged Data Platform
MapR Streams and MapR Converged Data PlatformMapR Streams and MapR Converged Data Platform
MapR Streams and MapR Converged Data Platform
 
Microservices in the Apache Kafka Ecosystem
Microservices in the Apache Kafka EcosystemMicroservices in the Apache Kafka Ecosystem
Microservices in the Apache Kafka Ecosystem
 
Self-Service Data Science for Leveraging ML & AI on All of Your Data
Self-Service Data Science for Leveraging ML & AI on All of Your DataSelf-Service Data Science for Leveraging ML & AI on All of Your Data
Self-Service Data Science for Leveraging ML & AI on All of Your Data
 
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...
Rethinking Stream Processing with Apache Kafka: Applications vs. Clusters, St...
 
Processing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaProcessing IoT Data with Apache Kafka
Processing IoT Data with Apache Kafka
 
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017
Introducing Apache Kafka's Streams API - Kafka meetup Munich, Jan 25 2017
 
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision Tree
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision TreeApache Kafka vs RabbitMQ: Fit For Purpose / Decision Tree
Apache Kafka vs RabbitMQ: Fit For Purpose / Decision Tree
 
Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015
 
Automatic Detection, Classification and Authorization of Sensitive Personal D...
Automatic Detection, Classification and Authorization of Sensitive Personal D...Automatic Detection, Classification and Authorization of Sensitive Personal D...
Automatic Detection, Classification and Authorization of Sensitive Personal D...
 
Best Practices for Enterprise User Management in Hadoop Environment
Best Practices for Enterprise User Management in Hadoop EnvironmentBest Practices for Enterprise User Management in Hadoop Environment
Best Practices for Enterprise User Management in Hadoop Environment
 
Apache Hadoop YARN: Past, Present and Future
Apache Hadoop YARN: Past, Present and FutureApache Hadoop YARN: Past, Present and Future
Apache Hadoop YARN: Past, Present and Future
 
File Format Benchmark - Avro, JSON, ORC and Parquet
File Format Benchmark - Avro, JSON, ORC and ParquetFile Format Benchmark - Avro, JSON, ORC and Parquet
File Format Benchmark - Avro, JSON, ORC and Parquet
 
Solving Cyber at Scale
Solving Cyber at ScaleSolving Cyber at Scale
Solving Cyber at Scale
 
Big Data in Azure
Big Data in AzureBig Data in Azure
Big Data in Azure
 
MaaS (Model as a Service): Modern Streaming Data Science with Apache Metron
MaaS (Model as a Service): Modern Streaming Data Science with Apache MetronMaaS (Model as a Service): Modern Streaming Data Science with Apache Metron
MaaS (Model as a Service): Modern Streaming Data Science with Apache Metron
 
Tuning kafka pipelines
Tuning kafka pipelinesTuning kafka pipelines
Tuning kafka pipelines
 
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...
Dancing Elephants - Efficiently Working with Object Stories from Apache Spark...
 

Similar to Apache Kafka Best Practices

Luxun a Persistent Messaging System Tailored for Big Data Collecting & Analytics
Luxun a Persistent Messaging System Tailored for Big Data Collecting & AnalyticsLuxun a Persistent Messaging System Tailored for Big Data Collecting & Analytics
Luxun a Persistent Messaging System Tailored for Big Data Collecting & AnalyticsWilliam Yang
 
Apache Hadoop 3.0 Community Update
Apache Hadoop 3.0 Community UpdateApache Hadoop 3.0 Community Update
Apache Hadoop 3.0 Community UpdateDataWorks Summit
 
SUSE Expert Days 2017 FUJITSU
SUSE Expert Days 2017 FUJITSUSUSE Expert Days 2017 FUJITSU
SUSE Expert Days 2017 FUJITSUSUSE España
 
Operating and supporting HBase Clusters
Operating and supporting HBase ClustersOperating and supporting HBase Clusters
Operating and supporting HBase Clustersenissoz
 
Operating and Supporting Apache HBase Best Practices and Improvements
Operating and Supporting Apache HBase Best Practices and ImprovementsOperating and Supporting Apache HBase Best Practices and Improvements
Operating and Supporting Apache HBase Best Practices and ImprovementsDataWorks Summit/Hadoop Summit
 
3PAR and VMWare
3PAR and VMWare3PAR and VMWare
3PAR and VMWarevmug
 
Membase Meetup 2010
Membase Meetup 2010Membase Meetup 2010
Membase Meetup 2010Membase
 
Meet HBase 2.0 and Phoenix 5.0
Meet HBase 2.0 and Phoenix 5.0Meet HBase 2.0 and Phoenix 5.0
Meet HBase 2.0 and Phoenix 5.0DataWorks Summit
 
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...GeeksLab Odessa
 
HDFS- What is New and Future
HDFS- What is New and FutureHDFS- What is New and Future
HDFS- What is New and FutureDataWorks Summit
 
Increasing Business Value Through High-Availability Technology
Increasing Business Value Through High-Availability TechnologyIncreasing Business Value Through High-Availability Technology
Increasing Business Value Through High-Availability TechnologyNexenta Systems
 
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea webhostingguy
 
Oracle rac 10g best practices
Oracle rac 10g best practicesOracle rac 10g best practices
Oracle rac 10g best practicesHaseeb Alam
 
Study of various factors affecting performance of multi core processors
Study of various factors affecting performance of multi core processorsStudy of various factors affecting performance of multi core processors
Study of various factors affecting performance of multi core processorsateeq ateeq
 

Similar to Apache Kafka Best Practices (20)

Open ebs 101
Open ebs 101Open ebs 101
Open ebs 101
 
LLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in HiveLLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in Hive
 
LLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in HiveLLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in Hive
 
LLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in HiveLLAP: Sub-Second Analytical Queries in Hive
LLAP: Sub-Second Analytical Queries in Hive
 
Hadoop 3 in a Nutshell
Hadoop 3 in a NutshellHadoop 3 in a Nutshell
Hadoop 3 in a Nutshell
 
Luxun a Persistent Messaging System Tailored for Big Data Collecting & Analytics
Luxun a Persistent Messaging System Tailored for Big Data Collecting & AnalyticsLuxun a Persistent Messaging System Tailored for Big Data Collecting & Analytics
Luxun a Persistent Messaging System Tailored for Big Data Collecting & Analytics
 
optimizing_ceph_flash
optimizing_ceph_flashoptimizing_ceph_flash
optimizing_ceph_flash
 
Apache Hadoop 3.0 Community Update
Apache Hadoop 3.0 Community UpdateApache Hadoop 3.0 Community Update
Apache Hadoop 3.0 Community Update
 
SUSE Expert Days 2017 FUJITSU
SUSE Expert Days 2017 FUJITSUSUSE Expert Days 2017 FUJITSU
SUSE Expert Days 2017 FUJITSU
 
Operating and supporting HBase Clusters
Operating and supporting HBase ClustersOperating and supporting HBase Clusters
Operating and supporting HBase Clusters
 
Operating and Supporting Apache HBase Best Practices and Improvements
Operating and Supporting Apache HBase Best Practices and ImprovementsOperating and Supporting Apache HBase Best Practices and Improvements
Operating and Supporting Apache HBase Best Practices and Improvements
 
3PAR and VMWare
3PAR and VMWare3PAR and VMWare
3PAR and VMWare
 
Membase Meetup 2010
Membase Meetup 2010Membase Meetup 2010
Membase Meetup 2010
 
Meet HBase 2.0 and Phoenix 5.0
Meet HBase 2.0 and Phoenix 5.0Meet HBase 2.0 and Phoenix 5.0
Meet HBase 2.0 and Phoenix 5.0
 
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
 
HDFS- What is New and Future
HDFS- What is New and FutureHDFS- What is New and Future
HDFS- What is New and Future
 
Increasing Business Value Through High-Availability Technology
Increasing Business Value Through High-Availability TechnologyIncreasing Business Value Through High-Availability Technology
Increasing Business Value Through High-Availability Technology
 
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea
Feb. 9, 2010 ICACT 2010@Phoenix Park, Korea
 
Oracle rac 10g best practices
Oracle rac 10g best practicesOracle rac 10g best practices
Oracle rac 10g best practices
 
Study of various factors affecting performance of multi core processors
Study of various factors affecting performance of multi core processorsStudy of various factors affecting performance of multi core processors
Study of various factors affecting performance of multi core processors
 

More from DataWorks Summit/Hadoop Summit

Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerDataWorks Summit/Hadoop Summit
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformDataWorks Summit/Hadoop Summit
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDataWorks Summit/Hadoop Summit
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...DataWorks Summit/Hadoop Summit
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...DataWorks Summit/Hadoop Summit
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLDataWorks Summit/Hadoop Summit
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)DataWorks Summit/Hadoop Summit
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...DataWorks Summit/Hadoop Summit
 

More from DataWorks Summit/Hadoop Summit (20)

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

Recently uploaded

TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityPrincipled Technologies
 
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...DianaGray10
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, AdobeApidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobeapidays
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)wesley chun
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Enterprise Knowledge
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...apidays
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdflior mazor
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherRemote DBA Services
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerThousandEyes
 
HTML Injection Attacks: Impact and Mitigation Strategies
HTML Injection Attacks: Impact and Mitigation StrategiesHTML Injection Attacks: Impact and Mitigation Strategies
HTML Injection Attacks: Impact and Mitigation StrategiesBoston Institute of Analytics
 
🐬 The future of MySQL is Postgres 🐘
🐬  The future of MySQL is Postgres   🐘🐬  The future of MySQL is Postgres   🐘
🐬 The future of MySQL is Postgres 🐘RTylerCroy
 
Real Time Object Detection Using Open CV
Real Time Object Detection Using Open CVReal Time Object Detection Using Open CV
Real Time Object Detection Using Open CVKhem
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptxHampshireHUG
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc
 
Artificial Intelligence: Facts and Myths
Artificial Intelligence: Facts and MythsArtificial Intelligence: Facts and Myths
Artificial Intelligence: Facts and MythsJoaquim Jorge
 
Developing An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilDeveloping An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilV3cube
 

Recently uploaded (20)

TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, AdobeApidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdf
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
HTML Injection Attacks: Impact and Mitigation Strategies
HTML Injection Attacks: Impact and Mitigation StrategiesHTML Injection Attacks: Impact and Mitigation Strategies
HTML Injection Attacks: Impact and Mitigation Strategies
 
🐬 The future of MySQL is Postgres 🐘
🐬  The future of MySQL is Postgres   🐘🐬  The future of MySQL is Postgres   🐘
🐬 The future of MySQL is Postgres 🐘
 
Real Time Object Detection Using Open CV
Real Time Object Detection Using Open CVReal Time Object Detection Using Open CV
Real Time Object Detection Using Open CV
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
Artificial Intelligence: Facts and Myths
Artificial Intelligence: Facts and MythsArtificial Intelligence: Facts and Myths
Artificial Intelligence: Facts and Myths
 
Developing An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilDeveloping An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of Brazil
 

Apache Kafka Best Practices

  • 2. 2 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Apache Kafka  Core APIs – The Producer API – The Consumer API – The Connector API – The Streams API  Broad classes of applications – Building real-time streaming data pipelines – Building real-time streaming applications – core building block in other data systems
  • 3. 3 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Key Concepts and Terminology
  • 4. 4 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Component Layout
  • 5. 5 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Hardware Guidance Cluster Size Memory CPU Storage Kafka Brokers 3+ 24G+ (for small) 64GB+ (for large) Multi- core processors( 12 CPU+ core), Hyper threading enabled 6+ x 1TB dedicated disks( RAID or JBOD) Zookeeper 3 (for small) 5 (for large) 8GB+ (for small) 24GB+ (for large) 2 core + SSD for Transaction logs
  • 6. 6 © Hortonworks Inc. 2011 – 2017. All Rights Reserved OS Tuning  OS Page Cache – Ex: Allocate to hold all the active segments of the log.  File descriptor limits : >100k  less swapping  Tcp tuning  JVM Configs – Java 8 with G1 Collector – 6-8 GB heap
  • 7. 7 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Disk Storage  Use multiple disk spindles, dedicated to kafka  JBOD vs RAID10  JBOD – Gives all the disk I/O  JBOD Limitations – any disk failure causes an unclean shutdown and requires lengthy recovery – data is not distributed consistently across disks – Multiple directories  KIP-112/113 – necessary tools for users to manage JBOD – Intelligent partition assignment – On disk failure, broker can serve replicas on the good disks – re-assign replicas between disks of the same broker
  • 8. 8 © Hortonworks Inc. 2011 – 2017. All Rights Reserved RAID  RAID10 – Can survive single disk failure – Performance and protection – balance load across disks – Single mount point – Performance hit and reduces the space  File System – EXT or XFS – SSD – Issues on NFS. – SAN, NAS
  • 9. 9 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Basic Monitoring  CPU Load  Network Metrics  File Handle Usage  Disk Space  Disk I/O Performance  Garbage Collection  ZooKeeper Monitoring
  • 10. 10 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Replication  Partition has replicas – Leader replica, Follower replicas  Leader maintains in-sync-replicas (ISR) – replica.lag.time.max.ms, num.replica.fetchers – min.insync.replica – used by producer to ensure greater durability https://www.slideshare.net/junrao/kafka-replication-apachecon2013
  • 11. 11 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Under Replicated Partitions  Number of partitions which are not fully replicated within the cluster  Mbean - kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions  ISR Shrink/Expand Rate  Under Replicated Partitions – Lost Broker? – Controller Issues – Zookeeper Issues – Network Issues  Solutions – Tune the ISR settings – Expand brokers
  • 12. 12 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Controller  Manages Partitions Life cycle  Avoid controller's ZK session expires – Soft failures – ISR Churn/Under replicated partitions – ZK Server performance – Long GC pauses on Broker – Bad network configuration  Monitoring – Mbean : kafka.controller:type=KafkaController,name=ActiveControllerCount – only one broker in the cluster should have 1 – LeaderElectionRate
  • 13. 13 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Unclean leader election  Enable replicas not in the ISR set to be elected as leader  Availability vs correctness – By-default kafka chooses availability  Monitoring – Mbean : kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec  Default will be changed in next release
  • 14. 14 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Broker Configs  log.retention.{ms, minutes, hours} , log.retention.bytes  message.max.bytes, replica.fetch.max.bytes  delete.topic.enable  unclean.leader.election.enable = false  min.insync.replicas = 2  replica.lag.time.max.ms, num.replica.fetchers  replica.fetch.response.max.bytes  zookeeper.session.timeout.ms = 30s  num.io.threads
  • 15. 15 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Cluster Sizing  Broker Sizing – Partition count on each broker (<2K) – Keep partition size on disk manageable (under 25GB per partition )  Cluster Size (no. of brokers) – how much retention we need – how much traffic cluster is getting  Cluster Expansion – Disk usage on the log segments partition should stay under 60% – Network usage on each broker should stay under 75%  Cluster Monitoring – Keep cluster balanced – Ensure that partitions of a topic are fairly distributed across brokers – Ensure that nodes in a cluster are not running out of disk and network
  • 16. 16 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Broker Monitoring  Partition Counts – Mbean: kafka.server:type=ReplicaManager,name=PartitionCount  Leader replica counts – Mbean: kafka.server:type=ReplicaManager,name=LeaderCount  ISR Shrink Rate/ISR expansion rate – kafka.server:type=ReplicaManager,name=IsrExpandsPerSec  Message in rate/Byte in rate/Byte out rate  NetworkProcessorAvgIdlePercent  RequestHandlerAvgIdlePercent
  • 17. 17 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Topic Sizing  No. of partitions – Have at least as many partitions as there are consumers in the largest group – topic is very busy – more partitions – Keep partition size on disk manageable (under 25GB per partition ) – Take into account any other application requirements – Special use cases – single partition  Keyed messages – enough partitions to deal with future growth  expanding partitions – whenever the size of the partition on disk is larger than threshold
  • 18. 18 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Choosing Partitions  Based on throughput requirements one can pick a rough number of partitions. – Lets call the throughput from producer to a single partition is P – Throughput from a single partition to a consumer is C – Target throughput is T – At least max (T/P, T/C)  More Partitions – More open file handles – May increase unavailability – May increase end-to-end latency – More memory for clients
  • 19. 19 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Quotas  Protect from bad clients and maintain SLAs  byte-rate thresholds on produce and fetch requests  can be applied to (user, client-id), user or client-id groups.  Server delays the responses  Broker Metrics for monitoring – throttle-rate, byte-rate  replica.fetch.response.max.bytes – Limit memory usage of replica fetch response  Limiting bandwidth usage during data migration – kafka-reassign-partitions.sh -- -throttle option
  • 20. 20 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Producer  User new java based clients  Test in your Environment – kafka-producer-perf-test.sh  Memory  CPU  Batch Compression  Avoid large messages – creates more memory pressure – slows down the brokers
  • 21. 21 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Critical Configs  batch.size – size based batching – larger size -> high throughput, higher latency  linger.ms – time based batching – larger size -> high throughput, higher latency  max.in.flight.requests.per.connection – Better throughput, affects ordering  compression.type – adding more user threads can help throughput  acks – Affects message durability
  • 22. 22 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Performance tuning  If throughput < network capacity – Add more user threads – Increase batch size – Add more producers instances – Add more partitions  Latency when acks = -1 – Increase num.replica.fetchers  Cross datacenter data transfer – Tune socket buffer settings, OS tcp buffer settings
  • 23. 23 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Producer Monitoring  batch-size-avg  compression-rate-avg  waiting-threads  buffer-available-bytes  record-queue-time-max  record-send-rate  records-per-request-avg
  • 24. 24 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Consumer  Test in your Environment – kafka-consumer-perf-test.sh  Throughput Issues – not enough partitions – OS Page Cache - allocate enough to hold all the messages for your consumers for say, 30s – Application/Processing logic  Offsets topic – __consumer_offsets – offsets.topic.replication.factor – offsets.retention.minutes – Monitor ISR, topic size  Slow offset commits – commit async, manual commits
  • 25. 25 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Consumer Configs  fetch.min.bytes and fetch.max.wait.ms  max.poll.interval.ms  max.poll.records  session.timeout.ms  Consumer Rebalance – check timeouts – check processing times/logic – GC Issues  Tune network settings
  • 26. 26 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Consumer Monitoring  Whether or not the consumer is keeping up with the messages that are being produced  Consumer Lag: Difference between the end of the log and the consumer offset  Monitoring – Metrics Monitoring - records-lag-max – bin/kafka-consumer-groups.sh – LinkedIn’s Burrow for consumer monitoring  Decreasing Lag – Analyze consumer - GC Issues, hung instance – Add more consumer Instances – increase the number of partitions and consumers
  • 27. 27 © Hortonworks Inc. 2011 – 2017. All Rights Reserved No data loss settings  Producer – block.on.buffer.full=true – retries=Long.MAX_VALUE – acks=all – max.in.flight.requests.per.connection=1 – close producer  Broker – replication factor >= 3 – min.insync.replicas=2 – disable unclean leader election  Consumer – disable auto.offset.commit – Commit offsets only after the messages are processed
  • 28. 28 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Authorizer - Ranger Auditing
  • 29. 29 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Mirror Maker  Tool to mirror a source Kafka cluster into a target (mirror) Kafka cluster
  • 30. 30 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Mirror Maker  Run multiple mirroring processes – high fault-tolerance – high throughput  --num.streams option to specify the number of consumer threads – no.of threads in num.streams
  • 31. 31 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Kafka Mirror Maker  Consumer and source cluster socket buffer sizes – high value for the socket buffer size – consumer's fetch size – OS networking Tuning  Source and Target Clusters are independent entities – Can be different numbers of partitions – offsets will not be the same. – partitioning order is preserved on a per-key basis.  Create topics in target cluster  Monitor whether a mirror is keeping up – Consumer Lag  Running In Secure Clusters – We recommend to use SSL – We can run MM on source cluster
  • 32. 32 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Open source Operational Tools  Ambari Metrics – https://docs.hortonworks.com/HDPDocuments/Ambari-2.4.2.0/bk_ambari-user- guide/content/grafana_kafka_dashboards.html  Removing brokers and rebalancing partitions in a cluster – https://github.com/linkedin/kafka-tools  Consumer Lag Monitoring – Burrow (https://github.com/linkedin/Burrow)  Kafka Manager - https://github.com/yahoo/kafka-manager
  • 33. 33 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Apache Kafka 0.10.2 release  Includes 15 KIPs, over 200 bug fixes and improvements  The newest Java Clients now support older brokers (0.10.0 and higher)  Separation of Internal and External traffic  Create Topic Policy  Security Improvements – Support for SASL/SCRAM mechanisms – Dynamic JAAS configuration for Kafka clients – Support for authentication of multiple Kafka clients in single JVM  Producer and Consumer Improvements  Connect API & Streams API improvements
  • 34. 34 © Hortonworks Inc. 2011 – 2017. All Rights Reserved Thank You
  • 35. 35 © Hortonworks Inc. 2011 – 2017. All Rights Reserved References  http://kafka.apache.org/documentation.html  https://community.hortonworks.com/articles/80813/kafka-best-practices-1.html  https://www.slideshare.net/JiangjieQin/producer-performance-tuning-for-apache- kafka-63147600  https://www.slideshare.net/ToddPalino/tuning-kafka-for-fun-and-profit  https://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka- 49753844  https://www.slideshare.net/ToddPalino/putting-kafka-into-overdrive  https://www.confluent.io/blog/how-to-choose-the-number-of-topicspartitions-in-a- kafka-cluster/