SlideShare a Scribd company logo
1 of 47
Apache Kafka®
and Stream
Processing at Pinterest
Liquan Pei, Boyang Chen, Shawn Nguyen
Agenda
What are we talking about today?
● Pinterest Overview
● Ads on Pinterest
● Ads Budgeting
● Ads Budgeting Kafka Client Migration
● Predictive Budgeting with Kafka Streams
Pinterest helps you discover
and do what you love
200M+ Monthly Active Users
100B+ Pins created by people
saving images from around
the web
2B+ ideas search monthly
A great platform for ads
Pinterest
Overview
Ads on Pinterest
Ads Product
What can advertisers get?
● CPC charge by clicks.
● CPM charge by impressions.
Ads Budgeting
● Prevent overdelivery.
● Pace advertiser spend throughout the billing cycle.
● To achieve the above goals, we need
realtime spend data.
Goals
Ads Budgeting
Overview
Ads Budgeting
Kafkaaction log
server log
Ads
Serving
Architecture
Budgeting Service
Kafka
Ads Budgeting
Kafkaaction log
server log
Ads
Serving
Architecture
Budgeting Service
Kafka
Ads Budgeting
● Every X minutes, the realtime spend information for all ads is published to S3.
● Every Y seconds, realtime spend information for ads that changes is published to
Kafka.
● Ads Servers tails Kafka to get the latest spend information.
● Each Kafka message is relatively big and the amount of data transferred via
network can be huge.
Workload
Ads Budgeting
Kafkaaction log
server log
Ads
Serving
Architecture 1.0
Budgeting Service
Kafka
Sarama Client
Sarama Client
Sarama Client
Sarama Go Client
● Ads Budgeting used a legacy Sarama Go Client to interact with Kafka.
● The legacy client had unexpected behavior when Kafka had failovers, high network
usage, or changes to ISRs.
○ Sarama Client can have inconsistent metadata information during broker leader
transitions, requiring service restarts to establish connections to the correct
brokers.
○ Metadata fetch requests occasionally hang.
Overview
Ads Budgeting
Kafkaaction log
server log
Ads
Serving
Architecture 1.0
Budgeting Service
Kafka
llbrdkafka
librdkafka
librdkafka
Zero Downtime Migration
● Introduce backward-compatible interfaces with Sarama.
● Introduce conversions between Sarama and Confluent’s go client event types.
● Introduce conversions between configurations for consumers and producers.
● Add switches in existing code to allow fast rollback within seconds.
● Lots of integration and unit tests.
Plan
Backward Compatible Interface
func (c *Client) Produce(topic string, partition int32, key, value []byte) error {
// create a librdkafka message struct
msg := ConvertedProducerMsg(topic, partition, key, value)
err = p.Produce(msg, p.Events())
if err != nil {
return err
}
// Check delivery report
e := <-p.Events()
m, ok := e.(*kafka.Message)
if !ok {
return fmt.Errorf("delivery report chan returned a non-message type: %s", m)
}
return nil
}
Producer
Backward Compatible Interface
func (c *Client) Consume(group string, topic string, partition int32, offset int64, conf
sarama.ConsumerConfig) (<-chan *sarama.ConsumerEvent, func(), error){
go func() {
run := true
for run == true {
select {
case ev := <-consumer.Events():
switch e := ev.(type) {
case *kafka.Message:
// convert Message to Sarama ConsumerEvent
consumeEvent := ConvertedConsumerEvent(e)
evCh <- consumeEvent
case kafka.Error:
…
}
}()
return evCh, func() { stopConsumer(consumer, evCh) }, err
Consumer
Metadata Management
// topicMetadata fetches all metadata info (leader/part isrs/etc.) for a given topic
func (c *Client) topicMetadata(topic string) ([]kafka.PartitionMetadata, error) {
// There is no exposed client interface in confluent-kafka other than consu
mer/producer
//Use a producer for now to send metadata requests
p, err := c.createDefaultProducer()
defer p.Close()
if err != nil {
return nil, err
}
metadata, err := p.GetMetadata(&topic, false, metadataTimeoutMs)
if err != nil {
return nil, err
}
return metadata.Topics[topic].Partitions, nil
}
Fetching Topic Metadata
Metadata Management
// FetchOffset fetches the requested earliest/latest offset for a given snapshot.
// Useful for bootstrapping from kafka as well as batch updates.
func (c *Client) FetchOffset(clientID, topic string, partition int32, method int) (int64,
error) {
if method != kafkasource.OffsetEarliest && method != kafkasource.OffsetLatest {
return -1, fmt.Errorf("bad offset type requested: %d", method)
}
p, err := c.createDefaultProducer()
defer p.Close()
if err != nil {
return -1, err
}
low, high, err := p.QueryWatermarkOffsets(topic, partition, metadataTimeoutMs)
…
}
Fetching Offsets for Batches
Putting new code in action doesn’t always go as expected.
Learnings
● Producer latency is higher than expected.
● Too many open sockets and threads.
● Which APIs to use?
● Broker network saturation.
Overview
Producer latency higher than expected
● Produce latency was around 75ms per message on sync producer (the
produce client acks delivery report synchronously per event before sending
subsequent records).
● Reduce queue.buffering.max.ms to 0ms (for sync producer, otherwise
msgs won’t send until buffer is filled or queue ms is reached)
Message Delays
Too many open sockets and threads
● File descriptor usage on brokers increased by over 10x.
● Redundant connections, N + len(bootstrap.servers), are created per client.
*10.1.239.229, and 10.1.225.203 were supplied in the client bootstrap list.
Resource Leak
Where did 10x sockets come from?
In an ideal world:
Partition Leader Replica Set
1 1 1,2,3
2 2 3,4,5
3 3 5,6,7
= 3 clients total w/ 1 client per partition = 3 threads and 3 sockets for consuming
In librdkafka (3 clients total and bootstrap.list containing brokers 1,2,3), we have:
Client A = 9 (1 socket per broker in cluster) + 3 (1 socket per broker in bootstrap list) = 12
Client B = 12 connections
Client C = 12 connections
= 36 connections total = ~10x the original connection count
Too many open sockets and threads
● Redundant connections, N + len(bootstrap.servers), are created per client if
broker’s ip/hostname in advertised.listeners do not match the exact
bootstrap broker name. The client compares this in the metadata response.
● See #LIBRDKAFKA-825 for fixes coming soon
○ Sparse connections/threads - Only connect and create threads to brokers we need to talk
to
○ Purge old brokers - Remove brokers no longer reported in metadata
Reducing socket usage
Which Consume API to use?
● We read directly from the events channel because there was slightly better
consume latency and reading from a chan is more natural in Go.
● Also, the poll method returns only 1 record per call, so reading from the Events
channel was simpler to express in code.
Broker network saturation
● With the default librdkafka consumer configuration, the network bandwidth
is reached during consumer initialization.
● We observed that the message consumption rate was constant.
● We changed the following configs to mitigate this issue
○ go.events.channel.size (max # msgs per consumer channel)
○ queued.min.messages (max # of msgs the consumer pre-fetches in the
background).
○ Enable compression on producers (snappy was used in our case).
Congestion Issue
Takeaways
● If sync producer is necessary and latency is critical, make sure buffering is as
minimal as possible.
● Tuning the buffers/channels and queue sizes are critical for achieving high
performance. In our app use case, pausing happens, making pre-fetching
unnecessary.
● Understand internals of consumer/producer APIs. This can uncover details
about performance and resource usages.
● Integration and performance tests are crucial! Performance is often
overlooked until code is deployed.
Librdkafka migration Q&A
● See https://github.com/edenhill/librdkafka/wiki/FAQ for more info and best
practices.
Ads Budgeting
Kafkaaction log
server log
Ads
Serving
Recap
Budgeting Service
Kafka
Advertiser X wants to do show ads on your website...
Advertiser Daily Budget Spend Impression
Spend per
impression
X $100 N/A 1000 $0.1
What is Overdelivery
What is Overdelivery
Advertiser Daily Budget Spend Impression
Spend per
impression
X $100 $100 1100 $0.09
Website
operator
Revenue
Internet
Company
N/A $100 1100 N/A
over deliver
100
The result was unexpected.
● Advertiser X earns more impressions.
● Solvable by realtime spend data.
What is Overdelivery
Advertiser Daily Budget Spend Impression
Spend per
impression
X $100 $100 1010 $0.1
Website
operator
Revenue
Internet
Company
N/A $100 1010 N/A
By making the system faster, overdelivery is reduced.
However, this is not the end of story...
Advertiser Y wants to do CPC (cost per click) ads on your website
Advertiser Daily Budget Spend Click Spend per click
Y $100 N/A 50 $2
Overdelivery continuing...
Advertiser Daily Budget Spend Click
Spend per
Click
Y $100 N/A 50 $2
Overdelivery continuing...
The result is unexpected, again.
● Advertiser Y earns more clicks.
● Because user action could be naturally delayed.
Advertiser Daily Budget Spend Click
Spend per
Click
Y $100 $100 60 $1.70
Website
operator
Revenue
Internet
Company
N/A $100 60 N/A
over deliver
10 clicks
The system needs to be able
to predict spend that might
occur in the future and slow
down campaigns close to
reaching their budget.
Ads Budgeting
Inflight Spend
Kafkaaction log
server log
Ads
Serving
Budgeting Service
Kafka
Inflight Spend
Service
Kafka
Inflight Spend
Methodology
● Predict potential spend based on previous 3 minute insertions. Output to
downstream every 10 seconds.
● Inflight_spend = price * impression_rate * action_rate
○ Price: the value of this ad
○ Impression_rate: historical conversion rate of one insertion to impression.
Note that an insertion is not guaranteed to convert to an impression.
○ Action_rate: for an advertiser paying by click, this is the probability that user
will click on this ad insertion; for advertiser paying by impression, this is 1.
Inflight Spend
Single Stage Aggregation
● We could use hopping windows to calculate the previous three minutes of expected
spend.
● Window could be three minutes long, spaced 10 seconds apart. There would be 180
seconds / 10 seconds = 18 open windows.
● However, each event may update all 18 windows.
● Write over read. (state store being written through Kafka)
Hopping window
Inflight Spend
Two stage aggregation
● Switched from large hopping windows to small tumbling windows.
● One 180 second overlapping window -> eighteen 10 second non-overlapping
window.
● Each event only updates one window at a time.
● By reducing the number of updates from 18 to 1, the overall throughput has
increased by 18x.
Tumbling window
Inflight Spend
Data publishing
Partition 1
Partition 2
Partition 4
Partition 3
Ads Serving
id spend
id spend
id spend
id spend
Message Map
● For each input partition, the lastest 3 minute inflight spend data is encoded as a
Kafka message.
● A map from ad ID to inflight spend by input partition.
● The client only needs to swap the snapshot in local.
Inflight Spend
Data encoding
● Since inflight spend accuracy is not strict we can use lookup table encoding(lossy)
for the data.
● Use delta encoding to encode ad IDs.
○ List of ad ids: [10000, 10002, 9999, 9980, 20000, 20010]
○ Sorted [9980, 9999, 10000, 10002, 20000, 20010]
○ Encoded list: [9980, 19, 1, 2, 20000, 10]
Inflight Spend
Delta encoding
Inflight Spend
3-minute wide time interval
Inflight vs Actual spend
Takeaways
● To make precise spend measurement, predictive approach needs to be taken.
● Think about whether read/write is more heavy when doing window design.
We are hiring!
● Ads Infrastructure:
○ Software Engineer, Ads Infrastructure
● Data:
○ Data Engineer, Logging Platform
○ Data Engineer, Stream Platform
○ Software Engineer, Big Data Platform
● Contact:
○ liquanpei@pinterest.com
○ bychen@pinterest.com
○ shnguyen@pinterest.com

More Related Content

More from confluent

Eventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkEventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkconfluent
 
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent CloudQ&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent Cloudconfluent
 
Citi TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep DiveCiti TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep Diveconfluent
 
Build real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with ConfluentBuild real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with Confluentconfluent
 
Q&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service MeshQ&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service Meshconfluent
 
Citi Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka MicroservicesCiti Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka Microservicesconfluent
 
Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3confluent
 
Citi Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging ModernizationCiti Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging Modernizationconfluent
 
Citi Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataCiti Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataconfluent
 
Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2confluent
 
Data In Motion Paris 2023
Data In Motion Paris 2023Data In Motion Paris 2023
Data In Motion Paris 2023confluent
 
Confluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with SynthesisConfluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with Synthesisconfluent
 
The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023confluent
 
The Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data StreamsThe Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data Streamsconfluent
 
The Journey to Data Mesh with Confluent
The Journey to Data Mesh with ConfluentThe Journey to Data Mesh with Confluent
The Journey to Data Mesh with Confluentconfluent
 
Citi Tech Talk: Monitoring and Performance
Citi Tech Talk: Monitoring and PerformanceCiti Tech Talk: Monitoring and Performance
Citi Tech Talk: Monitoring and Performanceconfluent
 
Confluent Partner Tech Talk with Reply
Confluent Partner Tech Talk with ReplyConfluent Partner Tech Talk with Reply
Confluent Partner Tech Talk with Replyconfluent
 
Citi Tech Talk Disaster Recovery Solutions Deep Dive
Citi Tech Talk  Disaster Recovery Solutions Deep DiveCiti Tech Talk  Disaster Recovery Solutions Deep Dive
Citi Tech Talk Disaster Recovery Solutions Deep Diveconfluent
 
Citi Tech Talk: Hybrid Cloud
Citi Tech Talk: Hybrid CloudCiti Tech Talk: Hybrid Cloud
Citi Tech Talk: Hybrid Cloudconfluent
 
Partner Tech Talk Q3: Q&A with PS - Migration and Upgrade
Partner Tech Talk Q3: Q&A with PS - Migration and UpgradePartner Tech Talk Q3: Q&A with PS - Migration and Upgrade
Partner Tech Talk Q3: Q&A with PS - Migration and Upgradeconfluent
 

More from confluent (20)

Eventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkEventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalk
 
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent CloudQ&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
 
Citi TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep DiveCiti TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep Dive
 
Build real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with ConfluentBuild real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with Confluent
 
Q&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service MeshQ&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service Mesh
 
Citi Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka MicroservicesCiti Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka Microservices
 
Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3
 
Citi Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging ModernizationCiti Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging Modernization
 
Citi Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataCiti Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time data
 
Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2
 
Data In Motion Paris 2023
Data In Motion Paris 2023Data In Motion Paris 2023
Data In Motion Paris 2023
 
Confluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with SynthesisConfluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with Synthesis
 
The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023
 
The Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data StreamsThe Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data Streams
 
The Journey to Data Mesh with Confluent
The Journey to Data Mesh with ConfluentThe Journey to Data Mesh with Confluent
The Journey to Data Mesh with Confluent
 
Citi Tech Talk: Monitoring and Performance
Citi Tech Talk: Monitoring and PerformanceCiti Tech Talk: Monitoring and Performance
Citi Tech Talk: Monitoring and Performance
 
Confluent Partner Tech Talk with Reply
Confluent Partner Tech Talk with ReplyConfluent Partner Tech Talk with Reply
Confluent Partner Tech Talk with Reply
 
Citi Tech Talk Disaster Recovery Solutions Deep Dive
Citi Tech Talk  Disaster Recovery Solutions Deep DiveCiti Tech Talk  Disaster Recovery Solutions Deep Dive
Citi Tech Talk Disaster Recovery Solutions Deep Dive
 
Citi Tech Talk: Hybrid Cloud
Citi Tech Talk: Hybrid CloudCiti Tech Talk: Hybrid Cloud
Citi Tech Talk: Hybrid Cloud
 
Partner Tech Talk Q3: Q&A with PS - Migration and Upgrade
Partner Tech Talk Q3: Q&A with PS - Migration and UpgradePartner Tech Talk Q3: Q&A with PS - Migration and Upgrade
Partner Tech Talk Q3: Q&A with PS - Migration and Upgrade
 

Recently uploaded

How we scaled to 80K users by doing nothing!.pdf
How we scaled to 80K users by doing nothing!.pdfHow we scaled to 80K users by doing nothing!.pdf
How we scaled to 80K users by doing nothing!.pdfSrushith Repakula
 
Intro in Product Management - Коротко про професію продакт менеджера
Intro in Product Management - Коротко про професію продакт менеджераIntro in Product Management - Коротко про професію продакт менеджера
Intro in Product Management - Коротко про професію продакт менеджераMark Opanasiuk
 
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...CzechDreamin
 
Demystifying gRPC in .Net by John Staveley
Demystifying gRPC in .Net by John StaveleyDemystifying gRPC in .Net by John Staveley
Demystifying gRPC in .Net by John StaveleyJohn Staveley
 
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdf
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdfLinux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdf
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdfFIDO Alliance
 
ECS 2024 Teams Premium - Pretty Secure
ECS 2024   Teams Premium - Pretty SecureECS 2024   Teams Premium - Pretty Secure
ECS 2024 Teams Premium - Pretty SecureFemke de Vroome
 
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...FIDO Alliance
 
Salesforce Adoption – Metrics, Methods, and Motivation, Antone Kom
Salesforce Adoption – Metrics, Methods, and Motivation, Antone KomSalesforce Adoption – Metrics, Methods, and Motivation, Antone Kom
Salesforce Adoption – Metrics, Methods, and Motivation, Antone KomCzechDreamin
 
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo Diehl
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo DiehlFuture Visions: Predictions to Guide and Time Tech Innovation, Peter Udo Diehl
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo DiehlPeter Udo Diehl
 
Connecting the Dots in Product Design at KAYAK
Connecting the Dots in Product Design at KAYAKConnecting the Dots in Product Design at KAYAK
Connecting the Dots in Product Design at KAYAKUXDXConf
 
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...CzechDreamin
 
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...FIDO Alliance
 
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)Julian Hyde
 
Enterprise Knowledge Graphs - Data Summit 2024
Enterprise Knowledge Graphs - Data Summit 2024Enterprise Knowledge Graphs - Data Summit 2024
Enterprise Knowledge Graphs - Data Summit 2024Enterprise Knowledge
 
Free and Effective: Making Flows Publicly Accessible, Yumi Ibrahimzade
Free and Effective: Making Flows Publicly Accessible, Yumi IbrahimzadeFree and Effective: Making Flows Publicly Accessible, Yumi Ibrahimzade
Free and Effective: Making Flows Publicly Accessible, Yumi IbrahimzadeCzechDreamin
 
Strategic AI Integration in Engineering Teams
Strategic AI Integration in Engineering TeamsStrategic AI Integration in Engineering Teams
Strategic AI Integration in Engineering TeamsUXDXConf
 
Google I/O Extended 2024 Warsaw
Google I/O Extended 2024 WarsawGoogle I/O Extended 2024 Warsaw
Google I/O Extended 2024 WarsawGDSC PJATK
 
Designing for Hardware Accessibility at Comcast
Designing for Hardware Accessibility at ComcastDesigning for Hardware Accessibility at Comcast
Designing for Hardware Accessibility at ComcastUXDXConf
 
The Metaverse: Are We There Yet?
The  Metaverse:    Are   We  There  Yet?The  Metaverse:    Are   We  There  Yet?
The Metaverse: Are We There Yet?Mark Billinghurst
 
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...FIDO Alliance
 

Recently uploaded (20)

How we scaled to 80K users by doing nothing!.pdf
How we scaled to 80K users by doing nothing!.pdfHow we scaled to 80K users by doing nothing!.pdf
How we scaled to 80K users by doing nothing!.pdf
 
Intro in Product Management - Коротко про професію продакт менеджера
Intro in Product Management - Коротко про професію продакт менеджераIntro in Product Management - Коротко про професію продакт менеджера
Intro in Product Management - Коротко про професію продакт менеджера
 
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...
Behind the Scenes From the Manager's Chair: Decoding the Secrets of Successfu...
 
Demystifying gRPC in .Net by John Staveley
Demystifying gRPC in .Net by John StaveleyDemystifying gRPC in .Net by John Staveley
Demystifying gRPC in .Net by John Staveley
 
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdf
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdfLinux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdf
Linux Foundation Edge _ Overview of FDO Software Components _ Randy at Intel.pdf
 
ECS 2024 Teams Premium - Pretty Secure
ECS 2024   Teams Premium - Pretty SecureECS 2024   Teams Premium - Pretty Secure
ECS 2024 Teams Premium - Pretty Secure
 
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...
ASRock Industrial FDO Solutions in Action for Industrial Edge AI _ Kenny at A...
 
Salesforce Adoption – Metrics, Methods, and Motivation, Antone Kom
Salesforce Adoption – Metrics, Methods, and Motivation, Antone KomSalesforce Adoption – Metrics, Methods, and Motivation, Antone Kom
Salesforce Adoption – Metrics, Methods, and Motivation, Antone Kom
 
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo Diehl
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo DiehlFuture Visions: Predictions to Guide and Time Tech Innovation, Peter Udo Diehl
Future Visions: Predictions to Guide and Time Tech Innovation, Peter Udo Diehl
 
Connecting the Dots in Product Design at KAYAK
Connecting the Dots in Product Design at KAYAKConnecting the Dots in Product Design at KAYAK
Connecting the Dots in Product Design at KAYAK
 
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...
SOQL 201 for Admins & Developers: Slice & Dice Your Org’s Data With Aggregate...
 
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...
FDO for Camera, Sensor and Networking Device – Commercial Solutions from VinC...
 
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)
Measures in SQL (a talk at SF Distributed Systems meetup, 2024-05-22)
 
Enterprise Knowledge Graphs - Data Summit 2024
Enterprise Knowledge Graphs - Data Summit 2024Enterprise Knowledge Graphs - Data Summit 2024
Enterprise Knowledge Graphs - Data Summit 2024
 
Free and Effective: Making Flows Publicly Accessible, Yumi Ibrahimzade
Free and Effective: Making Flows Publicly Accessible, Yumi IbrahimzadeFree and Effective: Making Flows Publicly Accessible, Yumi Ibrahimzade
Free and Effective: Making Flows Publicly Accessible, Yumi Ibrahimzade
 
Strategic AI Integration in Engineering Teams
Strategic AI Integration in Engineering TeamsStrategic AI Integration in Engineering Teams
Strategic AI Integration in Engineering Teams
 
Google I/O Extended 2024 Warsaw
Google I/O Extended 2024 WarsawGoogle I/O Extended 2024 Warsaw
Google I/O Extended 2024 Warsaw
 
Designing for Hardware Accessibility at Comcast
Designing for Hardware Accessibility at ComcastDesigning for Hardware Accessibility at Comcast
Designing for Hardware Accessibility at Comcast
 
The Metaverse: Are We There Yet?
The  Metaverse:    Are   We  There  Yet?The  Metaverse:    Are   We  There  Yet?
The Metaverse: Are We There Yet?
 
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...
Choosing the Right FDO Deployment Model for Your Application _ Geoffrey at In...
 

Apache Kafka® and Stream Processing at Pinterest

  • 1. Apache Kafka® and Stream Processing at Pinterest Liquan Pei, Boyang Chen, Shawn Nguyen
  • 2. Agenda What are we talking about today? ● Pinterest Overview ● Ads on Pinterest ● Ads Budgeting ● Ads Budgeting Kafka Client Migration ● Predictive Budgeting with Kafka Streams
  • 3. Pinterest helps you discover and do what you love
  • 4. 200M+ Monthly Active Users 100B+ Pins created by people saving images from around the web 2B+ ideas search monthly A great platform for ads Pinterest
  • 6. Ads Product What can advertisers get? ● CPC charge by clicks. ● CPM charge by impressions.
  • 7. Ads Budgeting ● Prevent overdelivery. ● Pace advertiser spend throughout the billing cycle. ● To achieve the above goals, we need realtime spend data. Goals
  • 9. Ads Budgeting Kafkaaction log server log Ads Serving Architecture Budgeting Service Kafka
  • 10. Ads Budgeting Kafkaaction log server log Ads Serving Architecture Budgeting Service Kafka
  • 11. Ads Budgeting ● Every X minutes, the realtime spend information for all ads is published to S3. ● Every Y seconds, realtime spend information for ads that changes is published to Kafka. ● Ads Servers tails Kafka to get the latest spend information. ● Each Kafka message is relatively big and the amount of data transferred via network can be huge. Workload
  • 12. Ads Budgeting Kafkaaction log server log Ads Serving Architecture 1.0 Budgeting Service Kafka Sarama Client Sarama Client Sarama Client
  • 13. Sarama Go Client ● Ads Budgeting used a legacy Sarama Go Client to interact with Kafka. ● The legacy client had unexpected behavior when Kafka had failovers, high network usage, or changes to ISRs. ○ Sarama Client can have inconsistent metadata information during broker leader transitions, requiring service restarts to establish connections to the correct brokers. ○ Metadata fetch requests occasionally hang. Overview
  • 14.
  • 15. Ads Budgeting Kafkaaction log server log Ads Serving Architecture 1.0 Budgeting Service Kafka llbrdkafka librdkafka librdkafka
  • 16. Zero Downtime Migration ● Introduce backward-compatible interfaces with Sarama. ● Introduce conversions between Sarama and Confluent’s go client event types. ● Introduce conversions between configurations for consumers and producers. ● Add switches in existing code to allow fast rollback within seconds. ● Lots of integration and unit tests. Plan
  • 17. Backward Compatible Interface func (c *Client) Produce(topic string, partition int32, key, value []byte) error { // create a librdkafka message struct msg := ConvertedProducerMsg(topic, partition, key, value) err = p.Produce(msg, p.Events()) if err != nil { return err } // Check delivery report e := <-p.Events() m, ok := e.(*kafka.Message) if !ok { return fmt.Errorf("delivery report chan returned a non-message type: %s", m) } return nil } Producer
  • 18. Backward Compatible Interface func (c *Client) Consume(group string, topic string, partition int32, offset int64, conf sarama.ConsumerConfig) (<-chan *sarama.ConsumerEvent, func(), error){ go func() { run := true for run == true { select { case ev := <-consumer.Events(): switch e := ev.(type) { case *kafka.Message: // convert Message to Sarama ConsumerEvent consumeEvent := ConvertedConsumerEvent(e) evCh <- consumeEvent case kafka.Error: … } }() return evCh, func() { stopConsumer(consumer, evCh) }, err Consumer
  • 19. Metadata Management // topicMetadata fetches all metadata info (leader/part isrs/etc.) for a given topic func (c *Client) topicMetadata(topic string) ([]kafka.PartitionMetadata, error) { // There is no exposed client interface in confluent-kafka other than consu mer/producer //Use a producer for now to send metadata requests p, err := c.createDefaultProducer() defer p.Close() if err != nil { return nil, err } metadata, err := p.GetMetadata(&topic, false, metadataTimeoutMs) if err != nil { return nil, err } return metadata.Topics[topic].Partitions, nil } Fetching Topic Metadata
  • 20. Metadata Management // FetchOffset fetches the requested earliest/latest offset for a given snapshot. // Useful for bootstrapping from kafka as well as batch updates. func (c *Client) FetchOffset(clientID, topic string, partition int32, method int) (int64, error) { if method != kafkasource.OffsetEarliest && method != kafkasource.OffsetLatest { return -1, fmt.Errorf("bad offset type requested: %d", method) } p, err := c.createDefaultProducer() defer p.Close() if err != nil { return -1, err } low, high, err := p.QueryWatermarkOffsets(topic, partition, metadataTimeoutMs) … } Fetching Offsets for Batches
  • 21. Putting new code in action doesn’t always go as expected.
  • 22. Learnings ● Producer latency is higher than expected. ● Too many open sockets and threads. ● Which APIs to use? ● Broker network saturation. Overview
  • 23. Producer latency higher than expected ● Produce latency was around 75ms per message on sync producer (the produce client acks delivery report synchronously per event before sending subsequent records). ● Reduce queue.buffering.max.ms to 0ms (for sync producer, otherwise msgs won’t send until buffer is filled or queue ms is reached) Message Delays
  • 24. Too many open sockets and threads ● File descriptor usage on brokers increased by over 10x. ● Redundant connections, N + len(bootstrap.servers), are created per client. *10.1.239.229, and 10.1.225.203 were supplied in the client bootstrap list. Resource Leak
  • 25. Where did 10x sockets come from? In an ideal world: Partition Leader Replica Set 1 1 1,2,3 2 2 3,4,5 3 3 5,6,7 = 3 clients total w/ 1 client per partition = 3 threads and 3 sockets for consuming In librdkafka (3 clients total and bootstrap.list containing brokers 1,2,3), we have: Client A = 9 (1 socket per broker in cluster) + 3 (1 socket per broker in bootstrap list) = 12 Client B = 12 connections Client C = 12 connections = 36 connections total = ~10x the original connection count
  • 26. Too many open sockets and threads ● Redundant connections, N + len(bootstrap.servers), are created per client if broker’s ip/hostname in advertised.listeners do not match the exact bootstrap broker name. The client compares this in the metadata response. ● See #LIBRDKAFKA-825 for fixes coming soon ○ Sparse connections/threads - Only connect and create threads to brokers we need to talk to ○ Purge old brokers - Remove brokers no longer reported in metadata Reducing socket usage
  • 27. Which Consume API to use? ● We read directly from the events channel because there was slightly better consume latency and reading from a chan is more natural in Go. ● Also, the poll method returns only 1 record per call, so reading from the Events channel was simpler to express in code.
  • 28. Broker network saturation ● With the default librdkafka consumer configuration, the network bandwidth is reached during consumer initialization. ● We observed that the message consumption rate was constant. ● We changed the following configs to mitigate this issue ○ go.events.channel.size (max # msgs per consumer channel) ○ queued.min.messages (max # of msgs the consumer pre-fetches in the background). ○ Enable compression on producers (snappy was used in our case). Congestion Issue
  • 29. Takeaways ● If sync producer is necessary and latency is critical, make sure buffering is as minimal as possible. ● Tuning the buffers/channels and queue sizes are critical for achieving high performance. In our app use case, pausing happens, making pre-fetching unnecessary. ● Understand internals of consumer/producer APIs. This can uncover details about performance and resource usages. ● Integration and performance tests are crucial! Performance is often overlooked until code is deployed.
  • 30. Librdkafka migration Q&A ● See https://github.com/edenhill/librdkafka/wiki/FAQ for more info and best practices.
  • 31. Ads Budgeting Kafkaaction log server log Ads Serving Recap Budgeting Service Kafka
  • 32. Advertiser X wants to do show ads on your website... Advertiser Daily Budget Spend Impression Spend per impression X $100 N/A 1000 $0.1 What is Overdelivery
  • 33. What is Overdelivery Advertiser Daily Budget Spend Impression Spend per impression X $100 $100 1100 $0.09 Website operator Revenue Internet Company N/A $100 1100 N/A over deliver 100 The result was unexpected. ● Advertiser X earns more impressions. ● Solvable by realtime spend data.
  • 34. What is Overdelivery Advertiser Daily Budget Spend Impression Spend per impression X $100 $100 1010 $0.1 Website operator Revenue Internet Company N/A $100 1010 N/A By making the system faster, overdelivery is reduced. However, this is not the end of story...
  • 35. Advertiser Y wants to do CPC (cost per click) ads on your website Advertiser Daily Budget Spend Click Spend per click Y $100 N/A 50 $2 Overdelivery continuing... Advertiser Daily Budget Spend Click Spend per Click Y $100 N/A 50 $2
  • 36. Overdelivery continuing... The result is unexpected, again. ● Advertiser Y earns more clicks. ● Because user action could be naturally delayed. Advertiser Daily Budget Spend Click Spend per Click Y $100 $100 60 $1.70 Website operator Revenue Internet Company N/A $100 60 N/A over deliver 10 clicks
  • 37. The system needs to be able to predict spend that might occur in the future and slow down campaigns close to reaching their budget.
  • 38. Ads Budgeting Inflight Spend Kafkaaction log server log Ads Serving Budgeting Service Kafka Inflight Spend Service Kafka
  • 39. Inflight Spend Methodology ● Predict potential spend based on previous 3 minute insertions. Output to downstream every 10 seconds. ● Inflight_spend = price * impression_rate * action_rate ○ Price: the value of this ad ○ Impression_rate: historical conversion rate of one insertion to impression. Note that an insertion is not guaranteed to convert to an impression. ○ Action_rate: for an advertiser paying by click, this is the probability that user will click on this ad insertion; for advertiser paying by impression, this is 1.
  • 40. Inflight Spend Single Stage Aggregation ● We could use hopping windows to calculate the previous three minutes of expected spend. ● Window could be three minutes long, spaced 10 seconds apart. There would be 180 seconds / 10 seconds = 18 open windows. ● However, each event may update all 18 windows. ● Write over read. (state store being written through Kafka) Hopping window
  • 41. Inflight Spend Two stage aggregation ● Switched from large hopping windows to small tumbling windows. ● One 180 second overlapping window -> eighteen 10 second non-overlapping window. ● Each event only updates one window at a time. ● By reducing the number of updates from 18 to 1, the overall throughput has increased by 18x. Tumbling window
  • 42. Inflight Spend Data publishing Partition 1 Partition 2 Partition 4 Partition 3 Ads Serving id spend id spend id spend id spend Message Map ● For each input partition, the lastest 3 minute inflight spend data is encoded as a Kafka message. ● A map from ad ID to inflight spend by input partition. ● The client only needs to swap the snapshot in local.
  • 43. Inflight Spend Data encoding ● Since inflight spend accuracy is not strict we can use lookup table encoding(lossy) for the data. ● Use delta encoding to encode ad IDs. ○ List of ad ids: [10000, 10002, 9999, 9980, 20000, 20010] ○ Sorted [9980, 9999, 10000, 10002, 20000, 20010] ○ Encoded list: [9980, 19, 1, 2, 20000, 10]
  • 45. Inflight Spend 3-minute wide time interval Inflight vs Actual spend
  • 46. Takeaways ● To make precise spend measurement, predictive approach needs to be taken. ● Think about whether read/write is more heavy when doing window design.
  • 47. We are hiring! ● Ads Infrastructure: ○ Software Engineer, Ads Infrastructure ● Data: ○ Data Engineer, Logging Platform ○ Data Engineer, Stream Platform ○ Software Engineer, Big Data Platform ● Contact: ○ liquanpei@pinterest.com ○ bychen@pinterest.com ○ shnguyen@pinterest.com