Embedded Mirror Maker
Transcript of Embedded Mirror Maker
Embedded Mirror Maker
Simon Suo @ LinkedInStreams - Weekly Deep Dive - April 18, 2016
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 MachinesAcross 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
MMProducerSource 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
ScenariosLeader movement on source cluster
Leader movement on destination cluster
Partition expansion
Topic creation
CaveatsMessage 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 = 0val MagicValue_V1: Byte = 1val 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 setOuter: | 4
| 7 |10 |
Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0 | 1 | 2 |
Expected message setOuter: | 4
| 7 |10 |
Inner: | 0 | 1 | 2 | 3 | 4 | 0 | 1 | 2 | 0 | 1 | 2 |
Future workSupport custom partition assignment scheme
Measure and reduce latency
Per-topic configurations
Questions?
References
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+mirroring
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+mirroring+(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