SlideShare uma empresa Scribd logo
1 de 79
Baixar para ler offline
Penghui Li
Apache Pulsar PMC Member
Transaction Support in Pulsar
Yong Zhang
Apache Pulsar Contributor
What is Apache Pulsar?
Pub/Sub Messaging
“Flexible Pub/Sub messaging
backed by durable log/stream storage”
2012: Pulsar idea started at Yahoo!

5 years on production, 100+ applications, 10+ data centers
2016/09 Yahoo open sourced Pulsar

2017/06 Yahoo donated Pulsar to ASF

2018/09 Pulsar graduated as a Top-Level project

2018/09 InfoWorld Best Open Source Project
Pulsar Community
Pulsar Community
• At-most once
• At-least once
• Exactly once
Messaging Semantics
• At-most once
• At-least once
• Exactly once
Messaging Semantics
Before 1.20.0-incubating
• At-most once
• At-least once
• Exactly once
Messaging Semantics
PIP-6: Guaranteed Message Deduplication
Revisit Existing Semantics
Pulsar’s Existing Semantics
Log
BrokerProducer
send(m1)
Pulsar’s Existing Semantics
Log
BrokerProducer
append(m1)
Pulsar’s Existing Semantics
Log
BrokerProducer
m1
Pulsar’s Existing Semantics
Log
BrokerProducer
ack(m1)
m1
Pulsar’s Existing Semantics
Log
BrokerProducer
ack(m1)
m1
Pulsar’s Existing Semantics
Log
BrokerProducer
send(m2)
m1
Pulsar’s Existing Semantics
Log
BrokerProducer
append(m2)
m1
m2
Pulsar’s Existing Semantics
Log
BrokerProducer
m1
m2
ack(m2)
Pulsar’s Existing Semantics
Log
BrokerProducer
m1
m2
ack(m2)
What do we do now?
At Least Once
Log
BrokerProducer
m1
m2
send(m2)
At Least Once
Log
BrokerProducer
m1
m2
append(m2)
m2
At Least Once
Log
BrokerProducer
m1
m2
append(m2)
m2
Duplicates !!
• Broker can fail
• The request from Producer to Broker can fail
• Producer or Consumer can fail
Why the duplicates are introduced?
I want exactly-once
• Producer: Idempotent Producer
• Broker: Guaranteed Message Deduplication (PIP-6)
• Consumer: Reader + Checkpoints (Flink / Spark)
Message Deduplication
• Producer Name - Identify who is producing the messages
• Sequence ID - Identify the message
• Producer Name + Sequence ID: The unique identifier for a
message
Idempotent Producer
• Broker maintains a map between Producer Name and Last-
Produced-Sequence-ID
• Broker accepts messages if the sequence id of a new
message is larger than its last produced sequence id
• Broker treats messages whose sequence id are smaller
• Broker keeps the map in a de-duplication cursor (stored in
bookkeeper)
Guaranteed Message Deduplication
Exactly Once
Log
BrokerProducer
send(1, m1)
Exactly Once
Log
BrokerProducer
append(1, m1)
1,m1
Exactly Once
Log
BrokerProducer
append(2, m2)
1,m1
2,m2
Exactly Once
Log
BrokerProducer
1,m1
2,m2
ack(2, m2)
What do we do now?
Exactly Once
Log
BrokerProducer
1,m1
2,m2
send(2, m2)
Exactly Once
Log
BrokerProducer
1,m1
2,m2
append(2, m2)
Exactly Once
Log
BrokerProducer
1,m1
2,m2
append(2, m2)
Duplicate detected
Exactly Once
Log
BrokerProducer
1,m1
2,m2
ack(2, m2)
• `bin/pulsar-admin set-deduplication -e tenant/namespace`
• Set producer name when creating a Producer
• Specify increasing sequence id when producing messages
Enable Exactly Once
• It only works when producing messages to one partition
• It only works for producing one message
• There is no atomicity when producing multiple messages to
one partition or many partitions
• Consumers are required to store the MessageId along with
its state and seek back to the MessageId when restoring
the state
Limitations
Introducing Transactions
PulsarCash
PulsarCash
Transfer $10
Alice Bob
• Transfer Topic : record the transfer requests
• Cash Transfer Function: perform the cash transfer action
• BalanceUpdate Topic: record the balance-update requests
PulsarCash, powered by Apache Pulsar
PulsarCash
Cash Transfer
Function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)
Transfer Topic
Ack Transfer
Cash Transfer
function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)
Ack: (100,0,0)
Reprocessed Transfer!
Cash Transfer
function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)
Ack: (100,0,0)
Lost Money!
Cash Transfer
function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)
Ack: (100,0,0)
Pulsar Transaction Explained
• Atomic writes across multiple partitions
• Atomic acknowledges across multiple subscriptions
• All the actions made within one transaction either all
succeed or all fail
• Consumers are *ONLY* allowed to read committed
messages
Transaction Semantics
Message<String> message = inputConsumer.receive();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage().value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage().value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId());
Without Transaction API
Broker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Data Log Data Log
Pulsar Client
Input
Consumer
Producer 1 Producer 2
0) Receive Message
1) Produce Messages
2) Ack Messages
Message<String> message = inputConsumer.receive();
Transaction txn = client.newTransaction().withTransactionTimeout(…).build().get();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
txn.commit().get();
MessageId msgId1 = sendFuture1.get();
MessageId msgId2 = sendFuture2.get();
Transaction API
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
• TC: transaction manager, coordinating committing and
aborting transactions
• In-Memory + Transaction Log
• Transaction Log is powered by a partitioned Pulsar topic
• `pulsar/system/__transaction_coordinator_log`
• Locating a TC is locating a partition of the transaction log
topic
Transaction Coordinator (TC)
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
• TB: store and index transaction data per topic partition
• TB is implemented using another ML (managed-ledger) as
TB log
• Messages are appended to into TB log
• Transaction Index is maintained in memory and
snapshotted to ledgers
• Transaction Index can be replayed from TB log
Transaction Buffer (TB)
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
• Introduce ACK_PENDING state
• Add response for acknowledgement, aka ack-on-ack
• Ack state is updated to cursor ledger
• Ack state can be replayed from cursor ledger
Transactional Subscription State
Transaction Execution Flow
Message<String> message = inputConsumer.receive();
Transaction txn = client.newTransaction().withTransactionTimeout(…).build().get();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
txn.commit().get();
MessageId msgId1 = sendFuture1.get();
MessageId msgId2 = sendFuture2.get();
Transaction API - New Transaction
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
1. New Txn
Tx1
Message<String> message = inputConsumer.receive();
Transaction txn = client.newTransaction().withTransactionTimeout(…).build().get();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
txn.commit().get();
MessageId msgId1 = sendFuture1.get();
MessageId msgId2 = sendFuture2.get();
Transaction API - Produce Messages
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
2.0 Add Produced Topics
To Txn
Tx1
Tx1: add [T1, T2] Tx1: M1 Tx1: M2
2.1 Produced Messages
To Topics with Txn
Message<String> message = inputConsumer.receive();
Transaction txn = client.newTransaction().withTransactionTimeout(…).build().get();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
txn.commit().get();
MessageId msgId1 = sendFuture1.get();
MessageId msgId2 = sendFuture2.get();
Transaction API - Acknowledges
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
3.0 Add Acked Subscriptions To Txn
Tx1
Tx1: add [T1, T2] Tx1: M1 Tx1: M2
3.0 Ack messages with Txn
Tx1: ACK (M0)
Tx1: add [S0]
Message<String> message = inputConsumer.receive();
Transaction txn = client.newTransaction().withTransactionTimeout(…).build().get();
CompletableFuture<MessageId> sendFuture1 =
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
CompletableFuture<MessageId> sendFuture2 =
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
txn.commit().get();
MessageId msgId1 = sendFuture1.get();
MessageId msgId2 = sendFuture2.get();
Transaction API - Commit
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
4.0 Commit Txn
Tx1
Tx1: add [T1, T2] Tx1: M1 Tx1: M2
Tx1: ACK (M0)
Tx1: add [S0]
4.0 Committing Txn
Tx1: Committing
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
Tx1
Tx1: add [T1, T2] Tx1: M1 Tx1: M2
Tx1: ACK (M0)
Tx1: add [S0]
4.1.0 Commit Txn
On Topics
4.1.1 Commit Txn
On Subscriptions
Tx1 (c) Tx1 (c)
Tx1: Committing
Tx1: Committed
Tx1: Committed
CoordinatorBroker-0 Broker-1
InputTopic OutputTopic-1 OutputTopic-2
Cursor
Transaction Log
Data Log
Txn Buffer
Data Log
Txn Buffer
Pulsar Client
Input
Consumer
Producer 1 Producer 2
Txn
New Txn
Tx1
Tx1: add [T1, T2] Tx1: M1 Tx1: M2
Tx1: ACK (M0)
Tx1: add [S0]
Tx1: Committing
Tx1 (c) Tx1 (c)
Tx1: Committed
Tx1: Committed
4.2 Committed Txn
inputConsumer.receiveAsync().thenCompose(message -> {
return client.newTransaction().withTransactionTimeout(…).build().thenCompose(txn -> {
producer1.newMessage(txn).value(“output-message-1”).sendAsync();
producer2.newMessage(txn).value(“output-message-2”).sendAsync();
inputConsumer.acknowledgeAsync(message.getMessageId(), txn);
return txn.commit();
});
})
Transaction API - Async Example
PulsarCash
Cash Transfer
function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)Ack: (100,0,0)
PulsarCash
Cash Transfer
function
Balance
user:alice, debit($10)
balance
update
balance
update
user:bob, credit($10)
(100,0,0): transfer($10, alice -> bob)Ack: (100,0,0)
Transaction
Make Event Streaming
easy, simple, and reliable for everyone
Pulsar Transaction
Available to use in Pulsar 2.6.0
When is it available?
• Transaction support in other languages (e.g. C++, Go)
• Transaction in Pulsar Functions & Pulsar IO
• Transaction in Kafka-on-Pulsar (KOP)
• Transaction for Flink / Spark job
• Transaction for State storage in Pulsar Functions
• …
Roadmap
• Ivan Kelly
• Matteo Merli
• Jia Zhai
• Penghui Li
• Marvin Cai
• Yong Zhang
• … and many other Pulsar users & contributors
Credits
Wechat Subscription: ApachePulsar
Mailing Lists

dev@pulsar.apache.org, users@pulsar.apache.org
Slack

https://apache-pulsar.slack.com (#china)

register: https://apache-pulsar.herokuapp.com/
https://github.com/apache/pulsar
https://github.com/apache/bookkeeper
Thanks!
Penghui Li Yong Zhang

Mais conteúdo relacionado

Mais procurados

Introduction to AMQP Messaging with RabbitMQ
Introduction to AMQP Messaging with RabbitMQIntroduction to AMQP Messaging with RabbitMQ
Introduction to AMQP Messaging with RabbitMQDmitriy Samovskiy
 
Introduction To RabbitMQ
Introduction To RabbitMQIntroduction To RabbitMQ
Introduction To RabbitMQKnoldus Inc.
 
How to get along with HATEOAS without letting the bad guys steal your lunch?
How to get along with HATEOAS without letting the bad guys steal your lunch?How to get along with HATEOAS without letting the bad guys steal your lunch?
How to get along with HATEOAS without letting the bad guys steal your lunch?Graham Charters
 
Introduction to REST - API
Introduction to REST - APIIntroduction to REST - API
Introduction to REST - APIChetan Gadodia
 
Microservices With Istio Service Mesh
Microservices With Istio Service MeshMicroservices With Istio Service Mesh
Microservices With Istio Service MeshNatanael Fonseca
 
NATS for Modern Messaging and Microservices
NATS for Modern Messaging and Microservices NATS for Modern Messaging and Microservices
NATS for Modern Messaging and Microservices NATS
 
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud Gateway
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud GatewaySpring IO 2023 - Dynamic OpenAPIs with Spring Cloud Gateway
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud GatewayIván López Martín
 
An Introduction to OAuth 2
An Introduction to OAuth 2An Introduction to OAuth 2
An Introduction to OAuth 2Aaron Parecki
 
Openstack zun,virtual kubelet
Openstack zun,virtual kubeletOpenstack zun,virtual kubelet
Openstack zun,virtual kubeletChanyeol yoon
 
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説You_Kinjoh
 
Redis and Kafka - Advanced Microservices Design Patterns Simplified
Redis and Kafka - Advanced Microservices Design Patterns SimplifiedRedis and Kafka - Advanced Microservices Design Patterns Simplified
Redis and Kafka - Advanced Microservices Design Patterns SimplifiedAllen Terleto
 
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containers
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker ContainersKafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containers
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containersconfluent
 
Présentation docker et kubernetes
Présentation docker et kubernetesPrésentation docker et kubernetes
Présentation docker et kubernetesKiwi Backup
 
Inter-Process Communication in Microservices using gRPC
Inter-Process Communication in Microservices using gRPCInter-Process Communication in Microservices using gRPC
Inter-Process Communication in Microservices using gRPCShiju Varghese
 
How to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams SafeHow to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams Safeconfluent
 
Monitoring kubernetes with prometheus
Monitoring kubernetes with prometheusMonitoring kubernetes with prometheus
Monitoring kubernetes with prometheusBrice Fernandes
 

Mais procurados (20)

Introduction to AMQP Messaging with RabbitMQ
Introduction to AMQP Messaging with RabbitMQIntroduction to AMQP Messaging with RabbitMQ
Introduction to AMQP Messaging with RabbitMQ
 
Introduction To RabbitMQ
Introduction To RabbitMQIntroduction To RabbitMQ
Introduction To RabbitMQ
 
How to get along with HATEOAS without letting the bad guys steal your lunch?
How to get along with HATEOAS without letting the bad guys steal your lunch?How to get along with HATEOAS without letting the bad guys steal your lunch?
How to get along with HATEOAS without letting the bad guys steal your lunch?
 
Zuul @ Netflix SpringOne Platform
Zuul @ Netflix SpringOne PlatformZuul @ Netflix SpringOne Platform
Zuul @ Netflix SpringOne Platform
 
Introduction to REST - API
Introduction to REST - APIIntroduction to REST - API
Introduction to REST - API
 
Envoy and Kafka
Envoy and KafkaEnvoy and Kafka
Envoy and Kafka
 
Microservices With Istio Service Mesh
Microservices With Istio Service MeshMicroservices With Istio Service Mesh
Microservices With Istio Service Mesh
 
NATS for Modern Messaging and Microservices
NATS for Modern Messaging and Microservices NATS for Modern Messaging and Microservices
NATS for Modern Messaging and Microservices
 
Kafka/SMM Crash Course
Kafka/SMM Crash CourseKafka/SMM Crash Course
Kafka/SMM Crash Course
 
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud Gateway
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud GatewaySpring IO 2023 - Dynamic OpenAPIs with Spring Cloud Gateway
Spring IO 2023 - Dynamic OpenAPIs with Spring Cloud Gateway
 
An Introduction to OAuth 2
An Introduction to OAuth 2An Introduction to OAuth 2
An Introduction to OAuth 2
 
Openstack zun,virtual kubelet
Openstack zun,virtual kubeletOpenstack zun,virtual kubelet
Openstack zun,virtual kubelet
 
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説
HTML5と WebSocket / WebRTC / Web Audio API / WebGL 技術解説
 
Redis and Kafka - Advanced Microservices Design Patterns Simplified
Redis and Kafka - Advanced Microservices Design Patterns SimplifiedRedis and Kafka - Advanced Microservices Design Patterns Simplified
Redis and Kafka - Advanced Microservices Design Patterns Simplified
 
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containers
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker ContainersKafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containers
Kafka Summit SF 2017 - Best Practices for Running Kafka on Docker Containers
 
Présentation docker et kubernetes
Présentation docker et kubernetesPrésentation docker et kubernetes
Présentation docker et kubernetes
 
Inter-Process Communication in Microservices using gRPC
Inter-Process Communication in Microservices using gRPCInter-Process Communication in Microservices using gRPC
Inter-Process Communication in Microservices using gRPC
 
How to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams SafeHow to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams Safe
 
Monitoring kubernetes with prometheus
Monitoring kubernetes with prometheusMonitoring kubernetes with prometheus
Monitoring kubernetes with prometheus
 
Kafka slideshare
Kafka   slideshareKafka   slideshare
Kafka slideshare
 

Semelhante a Transaction preview of Apache Pulsar

Transaction Support in Pulsar 2.5.0
Transaction Support in Pulsar 2.5.0Transaction Support in Pulsar 2.5.0
Transaction Support in Pulsar 2.5.0StreamNative
 
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...StreamNative
 
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...LINE Corporation
 
Fast Streaming into Clickhouse with Apache Pulsar
Fast Streaming into Clickhouse with Apache PulsarFast Streaming into Clickhouse with Apache Pulsar
Fast Streaming into Clickhouse with Apache PulsarTimothy Spann
 
From a Kafkaesque Story to The Promised Land at LivePerson
From a Kafkaesque Story to The Promised Land at LivePersonFrom a Kafkaesque Story to The Promised Land at LivePerson
From a Kafkaesque Story to The Promised Land at LivePersonLivePerson
 
From a kafkaesque story to The Promised Land
From a kafkaesque story to The Promised LandFrom a kafkaesque story to The Promised Land
From a kafkaesque story to The Promised LandRan Silberman
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonTimothy Spann
 
How bol.com makes sense of its logs, using the Elastic technology stack.
How bol.com makes sense of its logs, using the Elastic technology stack.How bol.com makes sense of its logs, using the Elastic technology stack.
How bol.com makes sense of its logs, using the Elastic technology stack.Renzo Tomà
 
Fault Tolerance at Speed
Fault Tolerance at SpeedFault Tolerance at Speed
Fault Tolerance at SpeedC4Media
 
Scaling big with Apache Kafka
Scaling big with Apache KafkaScaling big with Apache Kafka
Scaling big with Apache KafkaNikolay Stoitsev
 
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...Trivadis
 
JConf.dev 2022 - Apache Pulsar Development 101 with Java
JConf.dev 2022 - Apache Pulsar Development 101 with JavaJConf.dev 2022 - Apache Pulsar Development 101 with Java
JConf.dev 2022 - Apache Pulsar Development 101 with JavaTimothy Spann
 
Reactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsReactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsKonrad Malawski
 
[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lakeTimothy Spann
 
Big mountain data and dev conference apache pulsar with mqtt for edge compu...
Big mountain data and dev conference   apache pulsar with mqtt for edge compu...Big mountain data and dev conference   apache pulsar with mqtt for edge compu...
Big mountain data and dev conference apache pulsar with mqtt for edge compu...Timothy Spann
 
F_1330_Narkhede_Kafka .pptx
F_1330_Narkhede_Kafka .pptxF_1330_Narkhede_Kafka .pptx
F_1330_Narkhede_Kafka .pptxNIMITJAIN71
 
Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Sangjin Lee
 
Timeline service V2 at the Hadoop Summit SJ 2016
Timeline service V2 at the Hadoop Summit SJ 2016Timeline service V2 at the Hadoop Summit SJ 2016
Timeline service V2 at the Hadoop Summit SJ 2016Vrushali Channapattan
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Evan Chan
 

Semelhante a Transaction preview of Apache Pulsar (20)

Transaction Support in Pulsar 2.5.0
Transaction Support in Pulsar 2.5.0Transaction Support in Pulsar 2.5.0
Transaction Support in Pulsar 2.5.0
 
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...
Exactly-Once Made Easy: Transactional Messaging in Apache Pulsar - Pulsar Sum...
 
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...
Building a company-wide data pipeline on Apache Kafka - engineering for 150 b...
 
Fast Streaming into Clickhouse with Apache Pulsar
Fast Streaming into Clickhouse with Apache PulsarFast Streaming into Clickhouse with Apache Pulsar
Fast Streaming into Clickhouse with Apache Pulsar
 
From a Kafkaesque Story to The Promised Land at LivePerson
From a Kafkaesque Story to The Promised Land at LivePersonFrom a Kafkaesque Story to The Promised Land at LivePerson
From a Kafkaesque Story to The Promised Land at LivePerson
 
From a kafkaesque story to The Promised Land
From a kafkaesque story to The Promised LandFrom a kafkaesque story to The Promised Land
From a kafkaesque story to The Promised Land
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with Python
 
How bol.com makes sense of its logs, using the Elastic technology stack.
How bol.com makes sense of its logs, using the Elastic technology stack.How bol.com makes sense of its logs, using the Elastic technology stack.
How bol.com makes sense of its logs, using the Elastic technology stack.
 
Fault Tolerance at Speed
Fault Tolerance at SpeedFault Tolerance at Speed
Fault Tolerance at Speed
 
Scaling big with Apache Kafka
Scaling big with Apache KafkaScaling big with Apache Kafka
Scaling big with Apache Kafka
 
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...
Trivadis TechEvent 2016 Apache Kafka - Scalable Massage Processing and more! ...
 
JConf.dev 2022 - Apache Pulsar Development 101 with Java
JConf.dev 2022 - Apache Pulsar Development 101 with JavaJConf.dev 2022 - Apache Pulsar Development 101 with Java
JConf.dev 2022 - Apache Pulsar Development 101 with Java
 
Reactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsReactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka Streams
 
[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake
 
Big mountain data and dev conference apache pulsar with mqtt for edge compu...
Big mountain data and dev conference   apache pulsar with mqtt for edge compu...Big mountain data and dev conference   apache pulsar with mqtt for edge compu...
Big mountain data and dev conference apache pulsar with mqtt for edge compu...
 
F_1330_Narkhede_Kafka .pptx
F_1330_Narkhede_Kafka .pptxF_1330_Narkhede_Kafka .pptx
F_1330_Narkhede_Kafka .pptx
 
Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)
 
Timeline service V2 at the Hadoop Summit SJ 2016
Timeline service V2 at the Hadoop Summit SJ 2016Timeline service V2 at the Hadoop Summit SJ 2016
Timeline service V2 at the Hadoop Summit SJ 2016
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015
 
Kafka Explainaton
Kafka ExplainatonKafka Explainaton
Kafka Explainaton
 

Mais de StreamNative

Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022StreamNative
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...StreamNative
 
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...StreamNative
 
Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...StreamNative
 
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022StreamNative
 
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022StreamNative
 
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...StreamNative
 
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...StreamNative
 
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022StreamNative
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...StreamNative
 
Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022StreamNative
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...StreamNative
 
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022StreamNative
 
Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022StreamNative
 
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022StreamNative
 
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022StreamNative
 
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022StreamNative
 
Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022StreamNative
 
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...StreamNative
 
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...StreamNative
 

Mais de StreamNative (20)

Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
 
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
 
Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...
 
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
 
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
 
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
 
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
 
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022
Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
 
Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
 
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
 
Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022
 
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
 
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
 
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
 
Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022
 
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
 
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
 

Último

Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Researchmichael115558
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxolyaivanovalion
 
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779Best VIP Call Girls Noida Sector 39 Call Me: 8448380779
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779Delhi Call girls
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionfulawalesam
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...amitlee9823
 
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...amitlee9823
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Delhi Call girls
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxMohammedJunaid861692
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz1
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxolyaivanovalion
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...shivangimorya083
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 

Último (20)

Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in  KishangarhDelhi 99530 vip 56974 Genuine Escort Service Call Girls in  Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Research
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptx
 
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779Best VIP Call Girls Noida Sector 39 Call Me: 8448380779
Best VIP Call Girls Noida Sector 39 Call Me: 8448380779
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interaction
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
 
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signals
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFx
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdf
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 

Transaction preview of Apache Pulsar