SlideShare a Scribd company logo
1 of 29
Embedded Mirror Maker
Simon Suo @ LinkedIn
Streams - Weekly Deep Dive - April 18,
Background
History
KAFKA-74 (Oct 2011):
Originally implemented
with embedded approach
KAFKA-249 (Apr 2012):
Deprecated and replaced by
standalone approach in
0.7.1
NOW (Apr 2016): Re-
visiting and prototyping
an embedded approach
What has changed?
412 Machines
Across 12 fabrics
Motivation
Save machines (412 dedicated machines across 26 fabrics)
Save network (Eliminate producer to destination cluster network utilization)
Reduced latency (Shorten processing and network time)
Reduced request load on destination cluster, equal request load on source
cluster (Eliminate produce requests)
Equal processing load on source and destination cluster
Enable dynamic configuration of topics to mirror
Drawback
Tighter coupling of server and mirror features:
- Broker vulnerable to errors thrown from mirror (need good isolation)
- Mirror deployment tied to broker deployment (more difficult to hotfix)
Have to pass in clunky consumer configurations if customization is required
(can be mitigated by dynamic configuration via Zookeeper)
More complex server and mirror code (prototype proves it to be not too bad)
High level approach
Idempotent producer and free exactly once
transfer
Improve latency by supporting pipelining
(especially cross geographic mirroring)
No polling (especially idle topics)
Immediate reaction to partition expansion
and topic deletion
Idempotence can be done at log level
Pipeline does not help much (with
throughput)
Polling traffic is cheap
Issue with automatic topic creation
Source
Cluster
Produce Destination
Cluster
Consume
Public interface
Static configuration
Dynamic configuration &
Admin commands via
Zookeeper
Static configuration
/** ********* Mirror configuration ***********/
val NumMirrorConsumersProp = "num.mirror.consumers"
val MirrorRefreshMetadataBackoffMsProp = "mirror.refresh.metadata.backoff.ms"
val MirrorOffsetCommitIntervalMsProp = "mirror.offset.commit.interval.ms"
val MirrorRequiredAcksProp = "mirror.required.acks"
val MirrorAppendMessageTimeoutMsProp = "mirror.append.message.timeout.ms"
val MirrorTopicMapProp = "mirror.topic.map"
/** ********* Mirror configuration ***********/
val NumMirrorConsumersDoc = "Number of mirror consumers to use per destination broker per source cluster."
val MirrorOffsetCommitIntervalMsDoc = "The interval in milliseconds that the mirror consumer threads will use to commit offsets."
val MirrorRefreshMetadataBackoffMsDoc = "The interval in milliseconds used by the mirror consumer manager to refresh metadata of both
source and destination cluster(s)"
val MirrorRequiredAcksDoc = "This value controls when a message set append is considered completed."
val MirrorAppendMessageTimeoutMsDoc = "The amount of time the broker will wait trying to append message sets before timing out."
val MirrorTopicMapDoc = "A list of topics that this cluster should be mirroring. The format is
SOURCE_BOOTSTRAP_SERVERS_0:TOPIC_PATTERN;SOURCE_BOOTSTRAP_SERVERS_1:TOPIC_PATTERN"
Dynamic configuration & admin commands
/mirror
/clusterId0
/brokerId0
/command
/clusterId1
/brokerId1
Persistent z-node: root level
Persistent z-node: per source cluster config
Persistent z-node: admin commands
Ephemeral z-node: per destination broker state
Data = {
“version”: “1.0”,
“sourceBootstrapServer”: “???”,
“topicPattern”: “???”,
“numConsumers”: “???”,
“requiredAcks”: “???”
}
Data = {
“version”: “1.0”,
“Command”: “pause|resume|shutdown|startup|restart”
}
Data = {
“version”: “1.0”,
“State”: “paused|running|stopped|error”
}
Demo
Setup:
Destination: Local 2-node cluster with
local zookeeper (gitli trunk)
Source: kafka.uniform(0.8.2.66) &
kafka.charlie(0.9.0.2)
Validation: Kafka monitor trunk
Scenarios:
- Clean shutdown broker
- Rolling bounce brokers
- Pause and resume mirror
- Restart mirror
Guarantee:
- Zero data loss
- Zero data duplication
Implementation
At a glance:
consumer/ConsumerConfig.java (2)
consumer/internals/Fetcher.java (24)
kafka/log/Log.scala (6)
kafka/message/ByteBufferMessageSet.scala (35)
kafka/mirror/MirrorConsumer.scala (345)
kafka/mirror/MirrorConsumerManager.scala (377)
kafka/mirror/MirrorConsumerThread.scala (294)
kafka/mirror/MirrorFetcher.scala (180)
kafka/mirror/MirrorManager.scala (45)
kafka/server/KafkaApis.scala (5)
kafka/server/KafkaConfig.scala (58)
kafka/server/KafkaServer.scala (11)
kafka/utils/ZkUtils.scala (4)
Original:
kafka/tools/MirrorMaker.scala (673)
Original implementation
MirrorMaker
MMThread
MMThread
MMThread
MMThread
MMConsumer
MMConsumer
MMConsumer
MMConsumer
MMProducer
Source
Cluster
Destination
Cluster
Dedicated Machines
Decompress Re-compress
Proposed implementation
Destination Cluster
KafkaServer
MirrorConsumerManager
ReplicaManager
Partition
Partition
Partition
Partition
Source
Cluster
Source
Cluster
MetadataRefreshThread MirrorConsumer
MirrorConsumerThread MirrorConsumer
MirrorConsumerThread MirrorConsumer
Destination
Zookeeper
MirrorManager
Deep dive
Core components:
Metadata refresh
Partition assignment
Fetching
Appending to log
Committing offsets
Metadata refresh: finite state machine
Normal
Updated
Outdated
Paused
MirrorClusterCommandListener:
Listen to Zookeeper data change
Commit offsets synchronously & assign new
partition map to MirrorConsumer
Partition map updated by
MetadataRefreshThread
periodically and upon
request
Caught not leader for partition or
unknown topic or partition error
from ReplicaManager
Request metadata refresh from
MirrorConsumerManager
Partition assignment: round-robin by leader
partition0
Source cluster
(only leader partitions)
partition1 partition2 partition3
partition4 partition5
Destination cluster
(only leader partitions)
partition0
partition2
partition1
partition3
broker0 broker1
Fetching: modified new consumer
KafkaConsumer
Fetcher<K,V>
MirrorFetcher
MirrorConsumer
ConsumerNetworkClientConsumerCoordinator
def poll(timeout: Long): Map[TopicPartition, ByteBufferMessageSet]public ConsumerRecords<K, V> poll(long timeout)
Appending to log
Append to log:
only if thread state is normal or pause (abort if
metadata outdated or updated)
Update appended offsets:
when required acks are fulfilled and received callback
from replica manager with no error (skip and request metadata
update if leadership has changed)
Committing offsets
Asynchronous:
Configuration offset commit interval (default to 60
seconds
Synchronous:
Prior to clean shutdown of mirror
Upon destination cluster leadership change
Scenarios
Leader movement on source
cluster
Leader movement on
destination cluster
Partition expansion
Topic creation
Caveats
Message format version &
timestamp
Message sets & offset
assignment
Message format version & timestamp
/**
* The "magic" value
* When magic value is 0, the message uses absolute
offset and does not have a timestamp field.
* When magic value is 1, the message uses relative
offset and has a timestamp field.
*/
val MagicValue_V0: Byte = 0
val MagicValue_V1: Byte = 1
val CurrentMagicValue: Byte = 1
/**
* This method validates the timestamps of a message.
* If the message is using create time, this method checks if it is
within acceptable range.
*/
private def validateTimestamp(message: Message,
now: Long,
timestampType: TimestampType,
timestampDiffMaxMs: Long) {
if (timestampType == TimestampType.CREATE_TIME &&
math.abs(message.timestamp - now) > timestampDiffMaxMs)
throw new InvalidTimestampException(...)
if (!mirrored && message.timestampType ==
TimestampType.LOG_APPEND_TIME)
throw new InvalidTimestampException(...)
}
Message sets & offset assignment
Issue: No in-place offset assignment and need recompression
Solution: Use split iterator to split received message sets
into singular message sets (only containing one outer
message)
Received message set
Outer: | 4
| 7 |
10 |
Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0
| 1 | 2 |
Expected message set
Outer: | 4
| 7 |
10 |
Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0
| 1 | 2 |
Future work
Support custom partition
assignment scheme
Measure and reduce latency
Per-topic configurations
Questions?
References
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+mirro
ring
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+mirro
ring+(MirrorMaker)
https://issues.apache.org/jira/browse/KAFKA-74
https://issues.apache.org/jira/browse/KAFKA-249
Number of Mirror Maker Machines
1 #!/bin/sh
2
3
4 TOTAL_MACHINES=0
5 NUM_FABRICS=0
6 for i in `eh -e '%fabrics'`;do
7 NUM_IN_FABRIC=`eh -e %%${i}.kafka-mirror-maker | grep -iv noclusterdef | wc -l`
8 if [ $NUM_IN_FABRIC -gt 0 ]; then
9 TOTAL_MACHINES=$((TOTAL_MACHINES + NUM_IN_FABRIC))
10 NUM_FABRICS=$((NUM_FABRICS + 1))
11 echo ${i}: $NUM_IN_FABRIC;
12 fi
13 done
14 echo There are $TOTAL_MACHINES machines in total across $NUM_FABRICS fabrics

More Related Content

What's hot

Real-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormReal-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormDavorin Vukelic
 
Akka Microservices Architecture And Design
Akka Microservices Architecture And DesignAkka Microservices Architecture And Design
Akka Microservices Architecture And DesignYaroslav Tkachenko
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaJoe Stein
 
Exactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka StreamsExactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka StreamsGuozhang Wang
 
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...DataStax
 
Streaming and Messaging
Streaming and MessagingStreaming and Messaging
Streaming and MessagingXin Wang
 
Kafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingKafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingYaroslav Tkachenko
 
Realtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQRealtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQXin Wang
 
Real-time streams and logs with Storm and Kafka
Real-time streams and logs with Storm and KafkaReal-time streams and logs with Storm and Kafka
Real-time streams and logs with Storm and KafkaAndrew Montalenti
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and howPetr Zapletal
 
Deploying Docker Containers at Scale with Mesos and Marathon
Deploying Docker Containers at Scale with Mesos and MarathonDeploying Docker Containers at Scale with Mesos and Marathon
Deploying Docker Containers at Scale with Mesos and MarathonDiscover Pinterest
 
Kafka meetup JP #3 - Engineering Apache Kafka at LINE
Kafka meetup JP #3 - Engineering Apache Kafka at LINEKafka meetup JP #3 - Engineering Apache Kafka at LINE
Kafka meetup JP #3 - Engineering Apache Kafka at LINEkawamuray
 
Actors or Not: Async Event Architectures
Actors or Not: Async Event ArchitecturesActors or Not: Async Event Architectures
Actors or Not: Async Event ArchitecturesYaroslav Tkachenko
 
Distributed Applications with Apache Zookeeper
Distributed Applications with Apache ZookeeperDistributed Applications with Apache Zookeeper
Distributed Applications with Apache ZookeeperAlex Ehrnschwender
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lightbend
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in KafkaJoel Koshy
 
Practicing Continuous Deployment
Practicing Continuous DeploymentPracticing Continuous Deployment
Practicing Continuous Deploymentzeeg
 
Streaming millions of Contact Center interactions in (near) real-time with Pu...
Streaming millions of Contact Center interactions in (near) real-time with Pu...Streaming millions of Contact Center interactions in (near) real-time with Pu...
Streaming millions of Contact Center interactions in (near) real-time with Pu...Frank Kelly
 

What's hot (20)

Real-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache StormReal-Time Streaming with Apache Spark Streaming and Apache Storm
Real-Time Streaming with Apache Spark Streaming and Apache Storm
 
Akka Microservices Architecture And Design
Akka Microservices Architecture And DesignAkka Microservices Architecture And Design
Akka Microservices Architecture And Design
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache Kafka
 
Exactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka StreamsExactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka Streams
 
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...
Tuning Speculative Retries to Fight Latency (Michael Figuiere, Minh Do, Netfl...
 
Streaming and Messaging
Streaming and MessagingStreaming and Messaging
Streaming and Messaging
 
Kafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingKafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processing
 
Realtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQRealtime Statistics based on Apache Storm and RocketMQ
Realtime Statistics based on Apache Storm and RocketMQ
 
Real-time streams and logs with Storm and Kafka
Real-time streams and logs with Storm and KafkaReal-time streams and logs with Storm and Kafka
Real-time streams and logs with Storm and Kafka
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and how
 
Deploying Docker Containers at Scale with Mesos and Marathon
Deploying Docker Containers at Scale with Mesos and MarathonDeploying Docker Containers at Scale with Mesos and Marathon
Deploying Docker Containers at Scale with Mesos and Marathon
 
Kafka meetup JP #3 - Engineering Apache Kafka at LINE
Kafka meetup JP #3 - Engineering Apache Kafka at LINEKafka meetup JP #3 - Engineering Apache Kafka at LINE
Kafka meetup JP #3 - Engineering Apache Kafka at LINE
 
Actors or Not: Async Event Architectures
Actors or Not: Async Event ArchitecturesActors or Not: Async Event Architectures
Actors or Not: Async Event Architectures
 
Apache Storm
Apache StormApache Storm
Apache Storm
 
Distributed Applications with Apache Zookeeper
Distributed Applications with Apache ZookeeperDistributed Applications with Apache Zookeeper
Distributed Applications with Apache Zookeeper
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
 
Apache Spark
Apache SparkApache Spark
Apache Spark
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in Kafka
 
Practicing Continuous Deployment
Practicing Continuous DeploymentPracticing Continuous Deployment
Practicing Continuous Deployment
 
Streaming millions of Contact Center interactions in (near) real-time with Pu...
Streaming millions of Contact Center interactions in (near) real-time with Pu...Streaming millions of Contact Center interactions in (near) real-time with Pu...
Streaming millions of Contact Center interactions in (near) real-time with Pu...
 

Viewers also liked

Multi tier, multi-tenant, multi-problem kafka
Multi tier, multi-tenant, multi-problem kafkaMulti tier, multi-tenant, multi-problem kafka
Multi tier, multi-tenant, multi-problem kafkaTodd Palino
 
Kinesis vs-kafka-and-kafka-deep-dive
Kinesis vs-kafka-and-kafka-deep-diveKinesis vs-kafka-and-kafka-deep-dive
Kinesis vs-kafka-and-kafka-deep-diveYifeng Jiang
 
Kafka at Scale: Multi-Tier Architectures
Kafka at Scale: Multi-Tier ArchitecturesKafka at Scale: Multi-Tier Architectures
Kafka at Scale: Multi-Tier ArchitecturesTodd Palino
 
Project Deimos
Project DeimosProject Deimos
Project DeimosSimon Suo
 
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.Suneet Grover
 
101 mistakes FINN.no has made with Kafka (Baksida meetup)
101 mistakes FINN.no has made with Kafka (Baksida meetup)101 mistakes FINN.no has made with Kafka (Baksida meetup)
101 mistakes FINN.no has made with Kafka (Baksida meetup)Henning Spjelkavik
 
Kafka 0.9, Things you should know
Kafka 0.9, Things you should knowKafka 0.9, Things you should know
Kafka 0.9, Things you should knowRatish Ravindran
 
Kafkaesque days at linked in in 2015
Kafkaesque days at linked in in 2015Kafkaesque days at linked in in 2015
Kafkaesque days at linked in in 2015Joel Koshy
 
Kafka replication apachecon_2013
Kafka replication apachecon_2013Kafka replication apachecon_2013
Kafka replication apachecon_2013Jun Rao
 
Putting Kafka Into Overdrive
Putting Kafka Into OverdrivePutting Kafka Into Overdrive
Putting Kafka Into OverdriveTodd Palino
 
Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Guido Schmutz
 
Securing Kafka
Securing Kafka Securing Kafka
Securing Kafka confluent
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...DataWorks Summit/Hadoop Summit
 
Kafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereKafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereGwen (Chen) Shapira
 
Building a Replicated Logging System with Apache Kafka
Building a Replicated Logging System with Apache KafkaBuilding a Replicated Logging System with Apache Kafka
Building a Replicated Logging System with Apache KafkaGuozhang Wang
 
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 kafkaJiangjie Qin
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Productionconfluent
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaJiangjie Qin
 

Viewers also liked (20)

Multi tier, multi-tenant, multi-problem kafka
Multi tier, multi-tenant, multi-problem kafkaMulti tier, multi-tenant, multi-problem kafka
Multi tier, multi-tenant, multi-problem kafka
 
Kinesis vs-kafka-and-kafka-deep-dive
Kinesis vs-kafka-and-kafka-deep-diveKinesis vs-kafka-and-kafka-deep-dive
Kinesis vs-kafka-and-kafka-deep-dive
 
Kafka at Scale: Multi-Tier Architectures
Kafka at Scale: Multi-Tier ArchitecturesKafka at Scale: Multi-Tier Architectures
Kafka at Scale: Multi-Tier Architectures
 
Project Deimos
Project DeimosProject Deimos
Project Deimos
 
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.
Apache Kafka Bay Area Sep Meetup - 24/7 Customer, Inc.
 
101 mistakes FINN.no has made with Kafka (Baksida meetup)
101 mistakes FINN.no has made with Kafka (Baksida meetup)101 mistakes FINN.no has made with Kafka (Baksida meetup)
101 mistakes FINN.no has made with Kafka (Baksida meetup)
 
Kafka 0.9, Things you should know
Kafka 0.9, Things you should knowKafka 0.9, Things you should know
Kafka 0.9, Things you should know
 
Kafkaesque days at linked in in 2015
Kafkaesque days at linked in in 2015Kafkaesque days at linked in in 2015
Kafkaesque days at linked in in 2015
 
Kafka replication apachecon_2013
Kafka replication apachecon_2013Kafka replication apachecon_2013
Kafka replication apachecon_2013
 
Putting Kafka Into Overdrive
Putting Kafka Into OverdrivePutting Kafka Into Overdrive
Putting Kafka Into Overdrive
 
Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !
 
Securing Kafka
Securing Kafka Securing Kafka
Securing Kafka
 
Reliable and Scalable Data Ingestion at Airbnb
Reliable and Scalable Data Ingestion at AirbnbReliable and Scalable Data Ingestion at Airbnb
Reliable and Scalable Data Ingestion at Airbnb
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
 
Kafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be thereKafka Reliability - When it absolutely, positively has to be there
Kafka Reliability - When it absolutely, positively has to be there
 
Building a Replicated Logging System with Apache Kafka
Building a Replicated Logging System with Apache KafkaBuilding a Replicated Logging System with Apache Kafka
Building a Replicated Logging System with Apache Kafka
 
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
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Production
 
Netflix Data Pipeline With Kafka
Netflix Data Pipeline With KafkaNetflix Data Pipeline With Kafka
Netflix Data Pipeline With Kafka
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 

Similar to Embedded Mirror Maker: Reducing 412 Machines Across 12 Fabrics

Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014Monal Daxini
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...DataWorks Summit/Hadoop Summit
 
Percona XtraDB 集群文档
Percona XtraDB 集群文档Percona XtraDB 集群文档
Percona XtraDB 集群文档YUCHENG HU
 
Cloudsim & Green Cloud
Cloudsim & Green CloudCloudsim & Green Cloud
Cloudsim & Green CloudNeda Maleki
 
Cloudsim & greencloud
Cloudsim & greencloud Cloudsim & greencloud
Cloudsim & greencloud nedamaleki87
 
High Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureHigh Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureDataStax Academy
 
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...Amazon Web Services
 
NET Systems Programming Learned the Hard Way.pptx
NET Systems Programming Learned the Hard Way.pptxNET Systems Programming Learned the Hard Way.pptx
NET Systems Programming Learned the Hard Way.pptxpetabridge
 
Jvm Performance Tunning
Jvm Performance TunningJvm Performance Tunning
Jvm Performance Tunningguest1f2740
 
Jvm Performance Tunning
Jvm Performance TunningJvm Performance Tunning
Jvm Performance TunningTerry Cho
 
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...Amazon Web Services
 
Improving app performance with Kotlin Coroutines
Improving app performance with Kotlin CoroutinesImproving app performance with Kotlin Coroutines
Improving app performance with Kotlin CoroutinesHassan Abid
 
Technical Overview of Apache Drill by Jacques Nadeau
Technical Overview of Apache Drill by Jacques NadeauTechnical Overview of Apache Drill by Jacques Nadeau
Technical Overview of Apache Drill by Jacques NadeauMapR Technologies
 
Deep Dive on Amazon EC2 instances
Deep Dive on Amazon EC2 instancesDeep Dive on Amazon EC2 instances
Deep Dive on Amazon EC2 instancesAmazon Web Services
 
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021StreamNative
 
DevOps Enabling Your Team
DevOps Enabling Your TeamDevOps Enabling Your Team
DevOps Enabling Your TeamGR8Conf
 
Microservices with Micronaut
Microservices with MicronautMicroservices with Micronaut
Microservices with MicronautQAware GmbH
 
Achieve big data analytic platform with lambda architecture on cloud
Achieve big data analytic platform with lambda architecture on cloudAchieve big data analytic platform with lambda architecture on cloud
Achieve big data analytic platform with lambda architecture on cloudScott Miao
 

Similar to Embedded Mirror Maker: Reducing 412 Machines Across 12 Fabrics (20)

Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
 
Percona XtraDB 集群文档
Percona XtraDB 集群文档Percona XtraDB 集群文档
Percona XtraDB 集群文档
 
Cloudsim & Green Cloud
Cloudsim & Green CloudCloudsim & Green Cloud
Cloudsim & Green Cloud
 
Typesafe spark- Zalando meetup
Typesafe spark- Zalando meetupTypesafe spark- Zalando meetup
Typesafe spark- Zalando meetup
 
Cloudsim & greencloud
Cloudsim & greencloud Cloudsim & greencloud
Cloudsim & greencloud
 
High Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureHigh Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & Azure
 
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
 
NET Systems Programming Learned the Hard Way.pptx
NET Systems Programming Learned the Hard Way.pptxNET Systems Programming Learned the Hard Way.pptx
NET Systems Programming Learned the Hard Way.pptx
 
Jvm Performance Tunning
Jvm Performance TunningJvm Performance Tunning
Jvm Performance Tunning
 
Jvm Performance Tunning
Jvm Performance TunningJvm Performance Tunning
Jvm Performance Tunning
 
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
SRV402 Deep Dive on Amazon EC2 Instances, Featuring Performance Optimization ...
 
Improving app performance with Kotlin Coroutines
Improving app performance with Kotlin CoroutinesImproving app performance with Kotlin Coroutines
Improving app performance with Kotlin Coroutines
 
Technical Overview of Apache Drill by Jacques Nadeau
Technical Overview of Apache Drill by Jacques NadeauTechnical Overview of Apache Drill by Jacques Nadeau
Technical Overview of Apache Drill by Jacques Nadeau
 
Deep Dive on Amazon EC2 instances
Deep Dive on Amazon EC2 instancesDeep Dive on Amazon EC2 instances
Deep Dive on Amazon EC2 instances
 
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021
No Surprises Geo Replication - Pulsar Virtual Summit Europe 2021
 
DevOps Enabling Your Team
DevOps Enabling Your TeamDevOps Enabling Your Team
DevOps Enabling Your Team
 
Microservices with Micronaut
Microservices with MicronautMicroservices with Micronaut
Microservices with Micronaut
 
Achieve big data analytic platform with lambda architecture on cloud
Achieve big data analytic platform with lambda architecture on cloudAchieve big data analytic platform with lambda architecture on cloud
Achieve big data analytic platform with lambda architecture on cloud
 
G pars
G parsG pars
G pars
 

Embedded Mirror Maker: Reducing 412 Machines Across 12 Fabrics

  • 1. Embedded Mirror Maker Simon Suo @ LinkedIn Streams - Weekly Deep Dive - April 18,
  • 3. History KAFKA-74 (Oct 2011): Originally implemented with embedded approach KAFKA-249 (Apr 2012): Deprecated and replaced by standalone approach in 0.7.1 NOW (Apr 2016): Re- visiting and prototyping an embedded approach
  • 6. Motivation Save machines (412 dedicated machines across 26 fabrics) Save network (Eliminate producer to destination cluster network utilization) Reduced latency (Shorten processing and network time) Reduced request load on destination cluster, equal request load on source cluster (Eliminate produce requests) Equal processing load on source and destination cluster Enable dynamic configuration of topics to mirror
  • 7. Drawback Tighter coupling of server and mirror features: - Broker vulnerable to errors thrown from mirror (need good isolation) - Mirror deployment tied to broker deployment (more difficult to hotfix) Have to pass in clunky consumer configurations if customization is required (can be mitigated by dynamic configuration via Zookeeper) More complex server and mirror code (prototype proves it to be not too bad)
  • 8. High level approach Idempotent producer and free exactly once transfer Improve latency by supporting pipelining (especially cross geographic mirroring) No polling (especially idle topics) Immediate reaction to partition expansion and topic deletion Idempotence can be done at log level Pipeline does not help much (with throughput) Polling traffic is cheap Issue with automatic topic creation Source Cluster Produce Destination Cluster Consume
  • 9. Public interface Static configuration Dynamic configuration & Admin commands via Zookeeper
  • 10. Static configuration /** ********* Mirror configuration ***********/ val NumMirrorConsumersProp = "num.mirror.consumers" val MirrorRefreshMetadataBackoffMsProp = "mirror.refresh.metadata.backoff.ms" val MirrorOffsetCommitIntervalMsProp = "mirror.offset.commit.interval.ms" val MirrorRequiredAcksProp = "mirror.required.acks" val MirrorAppendMessageTimeoutMsProp = "mirror.append.message.timeout.ms" val MirrorTopicMapProp = "mirror.topic.map" /** ********* Mirror configuration ***********/ val NumMirrorConsumersDoc = "Number of mirror consumers to use per destination broker per source cluster." val MirrorOffsetCommitIntervalMsDoc = "The interval in milliseconds that the mirror consumer threads will use to commit offsets." val MirrorRefreshMetadataBackoffMsDoc = "The interval in milliseconds used by the mirror consumer manager to refresh metadata of both source and destination cluster(s)" val MirrorRequiredAcksDoc = "This value controls when a message set append is considered completed." val MirrorAppendMessageTimeoutMsDoc = "The amount of time the broker will wait trying to append message sets before timing out." val MirrorTopicMapDoc = "A list of topics that this cluster should be mirroring. The format is SOURCE_BOOTSTRAP_SERVERS_0:TOPIC_PATTERN;SOURCE_BOOTSTRAP_SERVERS_1:TOPIC_PATTERN"
  • 11. Dynamic configuration & admin commands /mirror /clusterId0 /brokerId0 /command /clusterId1 /brokerId1 Persistent z-node: root level Persistent z-node: per source cluster config Persistent z-node: admin commands Ephemeral z-node: per destination broker state Data = { “version”: “1.0”, “sourceBootstrapServer”: “???”, “topicPattern”: “???”, “numConsumers”: “???”, “requiredAcks”: “???” } Data = { “version”: “1.0”, “Command”: “pause|resume|shutdown|startup|restart” } Data = { “version”: “1.0”, “State”: “paused|running|stopped|error” }
  • 12. Demo Setup: Destination: Local 2-node cluster with local zookeeper (gitli trunk) Source: kafka.uniform(0.8.2.66) & kafka.charlie(0.9.0.2) Validation: Kafka monitor trunk Scenarios: - Clean shutdown broker - Rolling bounce brokers - Pause and resume mirror - Restart mirror Guarantee: - Zero data loss - Zero data duplication
  • 13. Implementation At a glance: consumer/ConsumerConfig.java (2) consumer/internals/Fetcher.java (24) kafka/log/Log.scala (6) kafka/message/ByteBufferMessageSet.scala (35) kafka/mirror/MirrorConsumer.scala (345) kafka/mirror/MirrorConsumerManager.scala (377) kafka/mirror/MirrorConsumerThread.scala (294) kafka/mirror/MirrorFetcher.scala (180) kafka/mirror/MirrorManager.scala (45) kafka/server/KafkaApis.scala (5) kafka/server/KafkaConfig.scala (58) kafka/server/KafkaServer.scala (11) kafka/utils/ZkUtils.scala (4) Original: kafka/tools/MirrorMaker.scala (673)
  • 15. Proposed implementation Destination Cluster KafkaServer MirrorConsumerManager ReplicaManager Partition Partition Partition Partition Source Cluster Source Cluster MetadataRefreshThread MirrorConsumer MirrorConsumerThread MirrorConsumer MirrorConsumerThread MirrorConsumer Destination Zookeeper MirrorManager
  • 16. Deep dive Core components: Metadata refresh Partition assignment Fetching Appending to log Committing offsets
  • 17. Metadata refresh: finite state machine Normal Updated Outdated Paused MirrorClusterCommandListener: Listen to Zookeeper data change Commit offsets synchronously & assign new partition map to MirrorConsumer Partition map updated by MetadataRefreshThread periodically and upon request Caught not leader for partition or unknown topic or partition error from ReplicaManager Request metadata refresh from MirrorConsumerManager
  • 18. Partition assignment: round-robin by leader partition0 Source cluster (only leader partitions) partition1 partition2 partition3 partition4 partition5 Destination cluster (only leader partitions) partition0 partition2 partition1 partition3 broker0 broker1
  • 19. Fetching: modified new consumer KafkaConsumer Fetcher<K,V> MirrorFetcher MirrorConsumer ConsumerNetworkClientConsumerCoordinator def poll(timeout: Long): Map[TopicPartition, ByteBufferMessageSet]public ConsumerRecords<K, V> poll(long timeout)
  • 20. Appending to log Append to log: only if thread state is normal or pause (abort if metadata outdated or updated) Update appended offsets: when required acks are fulfilled and received callback from replica manager with no error (skip and request metadata update if leadership has changed)
  • 21. Committing offsets Asynchronous: Configuration offset commit interval (default to 60 seconds Synchronous: Prior to clean shutdown of mirror Upon destination cluster leadership change
  • 22. Scenarios Leader movement on source cluster Leader movement on destination cluster Partition expansion Topic creation
  • 23. Caveats Message format version & timestamp Message sets & offset assignment
  • 24. Message format version & timestamp /** * The "magic" value * When magic value is 0, the message uses absolute offset and does not have a timestamp field. * When magic value is 1, the message uses relative offset and has a timestamp field. */ val MagicValue_V0: Byte = 0 val MagicValue_V1: Byte = 1 val CurrentMagicValue: Byte = 1 /** * This method validates the timestamps of a message. * If the message is using create time, this method checks if it is within acceptable range. */ private def validateTimestamp(message: Message, now: Long, timestampType: TimestampType, timestampDiffMaxMs: Long) { if (timestampType == TimestampType.CREATE_TIME && math.abs(message.timestamp - now) > timestampDiffMaxMs) throw new InvalidTimestampException(...) if (!mirrored && message.timestampType == TimestampType.LOG_APPEND_TIME) throw new InvalidTimestampException(...) }
  • 25. Message sets & offset assignment Issue: No in-place offset assignment and need recompression Solution: Use split iterator to split received message sets into singular message sets (only containing one outer message) Received message set Outer: | 4 | 7 | 10 | Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0 | 1 | 2 | Expected message set Outer: | 4 | 7 | 10 | Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0 | 1 | 2 |
  • 26. Future work Support custom partition assignment scheme Measure and reduce latency Per-topic configurations
  • 29. Number of Mirror Maker Machines 1 #!/bin/sh 2 3 4 TOTAL_MACHINES=0 5 NUM_FABRICS=0 6 for i in `eh -e '%fabrics'`;do 7 NUM_IN_FABRIC=`eh -e %%${i}.kafka-mirror-maker | grep -iv noclusterdef | wc -l` 8 if [ $NUM_IN_FABRIC -gt 0 ]; then 9 TOTAL_MACHINES=$((TOTAL_MACHINES + NUM_IN_FABRIC)) 10 NUM_FABRICS=$((NUM_FABRICS + 1)) 11 echo ${i}: $NUM_IN_FABRIC; 12 fi 13 done 14 echo There are $TOTAL_MACHINES machines in total across $NUM_FABRICS fabrics

Editor's Notes

  1. Mirror maker is a tool developed for Kafka to copy data from one Kafka cluster to another. It is essentially a consumer/producer pair. It runs on dedicated machines. This allows us to aggregate data from multiple datacenters into one container.
  2. 4 years ago, LinkedIn runs a much smaller scale Kafka deployment. Cost in terms of hardware was not the highest priority. It was cleaner to decouple the mirror maker tool from core functionality. But that decision is near-sighted But since then, our physical footprint has exploded. Now we have to consider the financial impact of that decision.
  3. Eliminate redundant step of decompressing and parsing into records, directly append to log instead
  4. We propose to use the destination cluster zookeeper as a public interface to provide dynamic configuration and admin command functionalities.
  5. Talk about implementation choices
  6. I encountered several unexpected caveats while implementing the new design. I think they are interesting enough to share and everyone can get a deeper exposure to the inner working of Kafka server.