+ All Categories
Home > Software > Ebay kafka usage at ebay

Ebay kafka usage at ebay

Date post: 18-Feb-2017
Category:
Upload: thomas-alex
View: 140 times
Download: 0 times
Share this document with a friend
26
KAFKA USAGE AT EBAY CMS TRAFFIC DELIVERY PIPELINE ED DUDENHOEFER, SENIOR SOFTWARE ENGINEER TRAFFIC MESSAGING PLATFORM TEAM @ EBAY, INC. 7/7/2016 EBAY CONFIDENTIAL 1
Transcript

EBAY CONFIDENTIAL 1

KAFKA USAGE AT EBAYCMS TRAFFIC DELIVERY PIPELINE

ED DUDENHOEFER, SENIOR SOFTWARE ENGINEERTRAFFIC MESSAGING PLATFORM TEAM @ EBAY, INC.

7/7/2016

EBAY CONFIDENTIAL 2

OVERVIEW

At Ebay, we use Kafka to loosely couple our event reception and processing machine pools.I will describe our Kafka cluster and best practices we have used to construct our communications delivery pipeline that processes 100 million messages per day, and will enable us to horizontally scale to 300+ million in H2 2016.

7/7/2016

EBAY CONFIDENTIAL 3

AGENDA

• What the CMS Traffic Team does at Ebay• Why we use Kafka in our pipeline• Architecture• Kafka topology• Dispatch service (producer)• Optimizer service (consumer … and producer)• Summary• Q & A

7/7/2016

EBAY CONFIDENTIAL 4

WHAT THE CMS TRAFFIC TEAM DOES AT EBAYEbay site needs a system to generate online and workflow-triggered notifications

• Auctions, watches, seller item registration, account admin to name a few

Ebay needs a system to generate campaigns• Marketing audiences, seasonal ads

Ebay needs a system to generate dynamic content from user clicks on site pages or in emails• Trending items, live auction results, fresh up-to-date content

We are the communications platform used for these purposes• Author templates with our Message Studio• Batch/stream process simple JSON dispatches into delivered HTML• Open-time rendering for dynamic content delivery• In-house replacement for Emarsys

7/7/2016

EBAY CONFIDENTIAL 5

CMS TRAFFIC PLATFORM

Message Studio template authoring• Web UI for creating template layouts and parameter contracts• Templates are Freemarker (FTL) script components and Ebay service calls• Preview and debugging of templates with mock dispatches

Open-time rendering of templates into online embeddable contentBatch stream processing of templates into deliverable messages

• Dispatch service collects and routes incoming JSON dispatches• From site “triggers” whenever their workflows dictate (auctions, etc.)• From marketing campaign generators for large bursts• Endpoints for routing to site or marketing pipelines (i.e. Kafka topics)

• Optimizer service orchestrates processing of queued dispatches• Various auditing, DW and experimentation operations• Rendering via templates into HTML or other format• Deliver system handoff and acknowledgement

7/7/2016

EBAY CONFIDENTIAL 6

CMS TRAFFIC PLATFORM

Message Studio template authoring• Web UI for creating template layouts and parameter contracts• Templates are Freemarker (FTL) script components and Ebay service calls• Preview and debugging of templates with mock dispatches

Open-time rendering of templates into online embeddable contentBatch stream processing of templates into deliverable messages

• Dispatch service collects and routes incoming JSON dispatches• From site “triggers” whenever their workflows dictate (auctions, etc.)• From marketing campaign generators for large bursts• Endpoints for routing to site or marketing pipelines (i.e. Kafka topics)

• Optimizer service orchestrates processing of queued dispatches• Various auditing, DW and experimentation operations• Rendering via templates into HTML or other format• Deliver system handoff and acknowledgement

7/7/2016

EBAY CONFIDENTIAL 7

Data Centers

BATCH/STREAM PROCESSING ARCHITECTURE

Dispatcher

LVS

PHX

SLC

Dispatcher

Dispatcher

Routing- Data center- Topic- Partition

hash

SiteTopi

c

MktgTopic

RetryTopic

Kafka

Optimizer

Render

Delivery

. . .

. . .

7/7/2016

EBAY CONFIDENTIAL

Optimizer

OPTIMIZER PROCESSING ARCHITECTURE

Connector

Render

Delivery

7/7/2016

8

SiteTopic

Kafka

OptimizerBlocking

Queue

RetryTopic

Marketing

Topic

Message filter / digest

EBAY CONFIDENTIAL 9

WHY WE USE KAFKA

Need a fast, light-weight, high throughput serial queue• Zookeeper client dependency (in 0.8.x) is fine, we actually use it for cache poke notifications as well

“At least once” delivery of messages is fineProducer / consumer partition assignment modelCustom partitioning and consumer rebalancingOther systems• RabbitMQ: overkill for us• Flume: we want to pull messages via consumer not get them pushed at us• Storm: we don’t need filtering, query, etc• IBM MQ, xxxMQ, …: features we don’t need, many cost $$

7/7/2016

EBAY CONFIDENTIAL 10

KAFKA USAGE

Currently on 0.8.x server and client, migrating to 0.9 in H2 2016• mainly for better live partition rebalancing distribution

Topic retention is 3 days, capacity > 500 million dispatches @ 2k each• average dispatch lifetime, from trigger to delivery acks

< 20 sec for site workflow emails< 1 min for marketing bursts (obviously longer when millions sent at once to us)

Topic queue depth usually low• can be temporarily large during marketing bursts, up to a few million• Kafka excels at this scenario, with its consumer and partitioning scheme

No multi-tenancy so no topic-level auth, internal-use only with firewall and bastion protection

7/7/2016

Main topics: partitions=1400, replication factor=3

EBAY CONFIDENTIAL 11

KAFKA TOPOLOGY

Clusters• 3 Data Center clusters (SLC, LVS, PHX)• 5 Kafka/Zookeeper brokers per clusterTopics (cluster all have the same layout)• 2 main topics (can have more), partitions=1400, replication factor=3

- Topic pipeline and host pools can overlap, more on this later

• 1 shared error topic per cluster, partitions=700, replication factor=3Hosts• Red Hat Enterprise Linux Server 6.1, Xeon (2 CPUs, 40 cores), 128 Gb memory• 50 Gb disk for server, logs, state change, admin• 1 Tb disk for partitions

7/7/2016

EBAY CONFIDENTIAL 12

KAFKA SERVER.PROPERTIES – I/O

# The number of threads handling network requestsnum.network.threads=8# The number of threads doing disk I/Onum.io.threads=8# The send buffer (SO_SNDBUF) used by the socket serversocket.send.buffer.bytes=1048576# The receive buffer (SO_RCVBUF) used by the socket serversocket.receive.buffer.bytes=1048576# The maximum size of a request that the socket server will accept (protection against OOM)socket.request.max.bytes=104857600

7/7/2016

EBAY CONFIDENTIAL 13

KAFKA SERVER.PROPERTIES – REPLICA & SHUTDOWN########################## Replica Settings ########################### settings taken from the kafka documentation at kafka.apache.org.num.replica.fetchers=4replica.fetch.max.bytes=1048576replica.fetch.wait.max.ms=500replica.high.watermark.checkpoint.interval.ms=5000replica.socket.timeout.ms=30000replica.socket.receive.buffer.bytes=65536replica.lag.time.max.ms=10000replica.lag.max.messages=4000######################### Shutdown Settings ##########################controlled.shutdown.enable=truecontrolled.shutdown.max.retries=5controlled.shutdown.retry.backoff.ms=1000

7/7/2016

EBAY CONFIDENTIAL 14

KAFKA SERVER.PROPERTIES – LOG FLUSH & RETENTION

############################# Log Basics ############################## The number of logical partitions per topic per server. More partitions allow greater parallelism# for consumption, but also mean more files.num.partitions=10auto.create.topics.enable=false############################# Log Flush Policy ############################## The following configurations control the flush of data to disk.# There are a few important trade-offs here:# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash.# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency).# 3. Throughput: The flush is generally the most expensive operation.# The settings below allow one to configure the flush policy to flush data after a period of time or# every N messages (or both). This can be done globally and overridden on a per-topic basis.# The number of messages to accept before forcing a flush of data to disklog.flush.interval.messages=10000# The maximum amount of time a message can sit in a log before we force a flushlog.flush.interval.ms=1000

############################# Log Retention Policy ############################## The following configurations control the disposal of log segments. The policy can# be set to delete segments after a period of time, or after a given size has accumulated.# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens# from the end of the log.# The minimum age of a log file to be eligible for deletionlog.retention.hours=72log.roll.hours=24# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining# segments don't drop below log.retention.bytes.log.retention.bytes=419430400# The maximum size of a log segment file. When this size is reached a new log segment will be created.log.segment.bytes=209715200# The interval at which log segments are checked to see if they can be deleted according# to the retention policieslog.cleanup.interval.mins=30

7/7/2016

EBAY CONFIDENTIAL 15

KAFKA – OPERATIONBare metal vs managed VM tenancy• Both available at Ebay, we use bare metal and self-monitorAlerts from internal event monitoring and rules• Under replication of partitions• Partition queue depth > N (threshold depends on the nature of topic use)• Host health (ping, heartbeat, ruok sanity checks)Remediation• Restart Zookeeper and/or Kafka via upstart• Replace and auto-deploy if hardware failure (has not happened in my time here)• SSH!

7/7/2016

EBAY CONFIDENTIAL 16

BATCH PROCESSING - OVERVIEW

Pipeline of SOA services loosely coupled by kafka topic queues• Site and marketing topics are the initial message buffer queues• Error topic serves as the loopback retry buffer (more on this in Optimizer)

Processes 100-200 million messages/day• up to 300 million/day by end of 2016

Transforms JSON message parmeters into a rendered communication• usually HTML for email, Ebay MC and/or SMS delivery channels• but can be anything authored into the Freemarker template components

7/7/2016

EBAY CONFIDENTIAL 17

BATCH PROCESSING - STRATEGIES

Apache Spark vs Custom task pipeline• Spark not available when we started this system• Specific thread pool and Kafka consumer control

- Non-obvious mapping from our system to externalize it- Mixed pool uses, some hosts participate in more than one topic pipeline

Kafka 0.9+ log compaction investigation for state durability• Defer batch block commits, post delete markers until batch done• Bridge parts of our pipeline that can suffer durability loss

- BlockingQueue between consumer thread receive from Kafka, and processing thread submit- Rules processing engine (like Esper CEP) which combines and emits messages out-of-order

Out-of-order complications• We use extra topics for error loopback reprocessing

- Priority and date ranges could be handled by separate topics (we don’t currently dinstinguish)• Scheduled delivery (orchestration)

- Leverage Kafka partitioning for novel purposes like timestamp delivery windows (future)

7/7/2016

EBAY CONFIDENTIAL 18

BATCH PROCESSING – DISPATCH DEFINITIONA dispatch is our term for a hierarchical JSON message mapJSON allows for structure and good compression via protocol bufferLifetime consists of passing among service calls, and Kafka residence

• Best practice: keep uuid, audit and state metadata in the dispatch but distinct, to enable cross-chain timings, log correlation and other shared state artifacts

Has distinct user data vs metadata state• User parameters are the original triggered ones sent to us• Metadata parameters are additional state we use for internal tracking

Easy to use curl, Fiddler, Postman, etc for testing and debugging7/7/2016

EBAY CONFIDENTIAL 19

DISPATCH EXAMPLE (METADATA HIDDEN){ "parameters": { "entity.common.version": "1", "email.unique.id": "11212121212", "lastName": "BNSv1.2", "target.user.id": "1276847275", "target.user.login": “bar", "lost.item.title": "abc", "notification.pref.url": "abc", "bid.user.name.last": "ram", "lost.item.id": "250006309658", "see.more.url": "ABC", "sender.user.email": "[email protected]",

"target.user.country": "US", "target.user.email": “[email protected]", "base.created.at": "1388502045578", "taget.user.language": "en", "meta.mail.content.format": "BOTH", "sale.site.id": "0", "target.user.site.id": "0", "email.soj.tags": "12121212121", "email.treatment.id": "45454545", "email.experiment.id": "666666666666", "email.log.id": "777777777777777", "message.center.enabled" : "true", }}

7/7/2016

EBAY CONFIDENTIAL 20

BATCH PROCESSING - PROTOCOL BUFFERSFor serialization to Kafka topics• Handles dispatch

hierarchy• Translates enums• Google protoc.exe

option java_outer_classname = "DispatchProto";option optimize_for = SPEED;option java_generic_services = false;message DispatchMesg { required int32 version = 1 [default = 2]; repeated Pair = 2;optional Meta = 3;}message Pair {

required string name = 1; repeated string value = 2; optional bool isArray = 3;}message Meta { repeated Pair = 1; repeated Submap submap = 2;}message Submap { required string name = 1; repeated Pair = 2; repeated Submap = 3;}

7/7/2016

EBAY CONFIDENTIAL 21

BATCH PROCESSING – DISPATCH SERVICE

VIPs over 48 hosts, divided between 3 data centersChooses DC, based on active DC list hash (userId % activeDCs.size())

• Best practice: active DC list configured via bean for live update in emergency

Chooses topic pipeline, based on endpoint called• site traffic is OLTP and flows constantly,

marketing is batched in large bursts• Best practice: use separate topics to prevent

bursts from starving OLTP from sudden huge interleaves

Chooses topic partition, based on custom partitioner (userId % 1400 partitions)

• due to Kafka api, userId is the byte encoding of long as string

public class UserIdPartitioner implements Partitioner { … @Override public int partition(Object userId, int partitionCount) { Long id = Long.parseLong(new String((byte[])userId)); return (int) (id % (long) partitionCount); }}

7/7/2016

EBAY CONFIDENTIAL 22

BATCH PROCESSING – OPTIMIZER SERVICENo endpoints, driven only by Kafka topic consumer threads• 2 host pools: site and marketing (12 hosts/dc = 36 hosts each)• 150 main topic consumer threads/host = 4200 consumer threads• 1400 partitions/topic (each dc) = 5200 partitions/topic• Consumer/partitions: each pool is 5400 consumers/4200 partitionsScale out and partition auto-rebalancing is sufficient• No tuning for consumers mapped to partitions, taking into account host, dc, broker distribution or hotspots• Best practice: consumers/partitions padded 20-30% for thread saturation yet assign single partitions per thread in most

cases (even with host failures or rolling restarts, all consumers service 1 or 2 partitions)Throttling and backoff policy• Downstream load tolerance and failure profile directly dictates our policy

o Recoverable failures from downstream timeout or service unavailability are rebuffered in the retry topico Each rebuffer or drop failure causes incremental backoff in recycling the 2 processor threads

(more consecutive failures = longer backoff period before enabling the 2 processor threads again)• Best practice: each consumer thread has a BlockingQueue to feed processing threads, and recycles processor threads on a

progressive backoff, letting the BlockingQueue block its consumer thread if downstream needs to breathe7/7/2016

EBAY CONFIDENTIAL 23

SUMMARY

• Kafka is essential in loosely coupling our producers and consumers in simple at-least-once FIFO topic queues, with very high through-put and easy maintenance needs• Kafka’s partitioning and consumer model, and auto-rebalance

handling are major features which simplify system design and allow easy horizontal scaling (pay attention to #partitions vs #consumers)• Log flushing and under-replication of partitions are the 2 big metrics

to monitor well in our Kafka clusters7/7/2016

EBAY CONFIDENTIAL 24

FUTURE …

• Upgrading to Kafka 0.9+ has significantly improved auto-rebalancing, especially for broker startup / rolling restart scenarios, decoupling consumer clients completely from Zookeeper, and redesigned consumer client APIs.

• Using Kafka log compaction as delete markers, instead of advancing read cursor all the time (good for out-of-order processing, or simultaneous batch marking as processed)

• Esper CEP message transformation (also wrapped with the previous feature to record deletes whenever messages are emitted out-of-order by Esper rules)

• At-most-once guarantees are acceptable for our purposes, but can be improved over time to ~exactly-once by shoring up durability holes

• More real-time rendering scenarios …

7/7/2016

EBAY CONFIDENTIAL 25

KAFKA RESOURCES• Apache Kafka

http://kafka.apache.org/• Kafka releases and features roadmap

https://cwiki.apache.org/confluence/display/KAFKA/Index• Kafka data structure in Zookeeper

https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper• Log compaction and delete markers/journaling uses

https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction• 0.9 Kafka release

http://www.confluent.io/blog/apache-kafka-0.9-is-released• 0.9 detailed consumer design

https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+DesignTools• Yahoo’s Kafka Manager – a Kafka broker and topic management visualizer

https://github.com/yahoo/kafka-manager• Netflix’s Exhibitor – a Zookeeper visualizer

https://github.com/Netflix/exhibitor• Sematext SPM Performance Monitoring – supports Kafka clusters and traffic monitoring

https://sematext.com/blog/2015/02/10/kafka-0-8-2-monitoring/7/7/2016

EBAY CONFIDENTIAL 26

Questions?

7/7/2016


Recommended