SlideShare ist ein Scribd-Unternehmen logo
1 von 39
Downloaden Sie, um offline zu lesen
© 2019 SPLUNK INC.
The Next Generation
Messaging and Queuing
System
© 2019 SPLUNK INC.
Intro
Senior Principal Engineer - Splunk
Co-creator Apache Pulsar
Matteo Merli
Senior Director of Engineering - Splunk
Karthik Ramasamy
© 2019 SPLUNK INC.
Messaging and Streaming
© 2019 SPLUNK INC.
Messaging
Message passing between
components, application,
services
© 2019 SPLUNK INC.
Streaming
Analyze events that just
happened
© 2019 SPLUNK INC.
Messaging vs Streaming
2 worlds, 1 infra
© 2019 SPLUNK INC.
Use cases
● OLTP, Integration
● Main challenges:
○ Latency
○ Availability
○ Data durability
○ High level features
■ Routing, DLQ, delays, individual acks
● Real-time analytics
● Main challenges:
○ Throughput
○ Ordering
○ Stateful processing
○ Batch + Real-Time
Messaging Streaming
© 2019 SPLUNK INC.
Storage
Messaging
Compute
© 2019 SPLUNK INC.
Apache Pulsar
Data replicated
and synced to
disk
Durability
Low publish
latency of 5ms at
99pct
Low
Latency
Can reach 1.8 M
messages/s in a
single partition
High
Throughput
System is
available if any 2
nodes are up
High
Availability
Take advantage
of dynamic
cluster scaling in
cloud
environments
Cloud
Native
Flexible Pub-Sub and Compute backed by durable log storage
© 2019 SPLUNK INC.
Apache Pulsar
Support both
Topic & Queue
semantic in a
single model
Unified
messaging
model
Can support
millions of topics
Highly
Scalable
Lightweight
compute
framework based
on functions
Native
Compute
Supports multiple
users and
workloads in a
single cluster
Multi
Tenant
Out of box
support for
geographically
distributed
applications
Geo
Replication
Flexible Pub-Sub and Compute backed by durable log storage
© 2019 SPLUNK INC.
Apache Pulsar project in numbers
192
Contributors
30
Committers
100s
Adopters
4.6K
Github Stars
© 2019 SPLUNK INC.
Sample of Pulsar users and contributors
© 2019 SPLUNK INC.
Messaging Model
© 2019 SPLUNK INC.
Pulsar Client libraries
● Java — C++ — C — Python — Go — NodeJS — WebSocket APIs
● Partitioned topics
● Apache Kafka compatibility wrapper API
● Transparent batching and compression
● TLS encryption and authentication
● End-to-end encryption
© 2019 SPLUNK INC.
Architectural view
Separate layers between
brokers bookies
● Broker and bookies can
be added independently
● Traffic can be shifted very
quickly across brokers
● New bookies will ramp up
on traffic quickly
© 2019 SPLUNK INC.
Apache BookKeeper
● Low-latency durable writes
● Simple repeatable read
consistency
● Highly available
● Store many logs per node
● I/O Isolation
Replicated log storage
© 2019 SPLUNK INC.
Inside
BookKeeper
Storage optimized for
sequential & immutable data
● IO isolation between write and read
operations
● Does not rely on OS page cache
● Slow consumers won’t impact latency
● Very effective IO patterns:
○ Journal — append only and no reads
○ Storage device — bulk write and
sequential reads
● Number of files is independent from number
of topics
© 2019 SPLUNK INC.
Segment
Centric
Storage
In addition to partitioning, messages are stored
in segments (based on time and size)
Segments are independent from each others and
spread across all storage nodes
© 2019 SPLUNK INC.
Segments vs Partitions
© 2019 SPLUNK INC.
Tiered
Storage
Unlimited topic storage capacity
Achieves the true “stream-storage”: keep the raw
data forever in stream form
Extremely cost effective
© 2019 SPLUNK INC.
Schema Registry
Store information on the data structure — Stored in BookKeeper
Enforce data types on topic
Allow for compatible schema evolutions
© 2019 SPLUNK INC.
Schema Registry
● Integrated schema in API
● End-to-end type safety — Enforced in Pulsar broker
Producer<MyClass> producer = client
.newProducer(Schema.JSON(MyClass.class))
.topic("my-topic")
.create();
producer.send(new MyClass(1, 2));
Consumer<MyClass> consumer = client
.newConsumer(Schema.JSON(MyClass.class))
.topic("my-topic")
.subscriptionName("my-subscription")
.subscribe();
Message<MyClass> msg = consumer.receive();
Type Safe API
© 2019 SPLUNK INC.
Geo
Replication
Scalable asynchronous replication
Integrated in the broker message flow
Simple configuration to add/remove regions
© 2019 SPLUNK INC.
Replicated Subscriptions
● Consumption will restart close to where a consumer left off - Small amount of dups
● Implementation
○ Use markers injected into the data flow
○ Create a consistent snapshot of message ids across cluster
○ Establish a relationship: If consumed MA-1 in Cluster-A it must have consumed
MB-2 in Cluster-B
Migrate subscriptions across geo-replicated clusters
© 2019 SPLUNK INC.
Multi-Tenancy
● Authentication / Authorization / Namespaces / Admin APIs
● I/O Isolations between writes and reads
○ Provided by BookKeeper
○ Ensure readers draining backlog won’t affect publishers
● Soft isolation
○ Storage quotas — flow-control — back-pressure — rate limiting
● Hardware isolation
○ Constrain some tenants on a subset of brokers or bookies
A single Pulsar cluster supports multiple users and mixed workloads
© 2019 SPLUNK INC.
Lightweight Compute with
Pulsar Functions
© 2019 SPLUNK INC.
Pulsar Functions
© 2019 SPLUNK INC.
Pulsar Functions
● User supplied compute against a
consumed message
○ ETL, data enrichment, filtering, routing
● Simplest possible API
○ Use language specific “function” notation
○ No SDK required
○ SDK available for more advanced
features (state, metrics, logging, …)
● Language agnostic
○ Java, Python and Go
○ Easy to support more languages
● Pluggable runtime
○ Managed or manual deployment
○ Run as threads, processes or containers
in Kubernetes
© 2019 SPLUNK INC.
Pulsar Functions
def process(input):
return input + '!'
import java.util.function.Function;
public class ExclamationFunction
implements Function<String, String> {
@Override
public String apply(String input) {
return input + "!";
}
}
Python Java
Examples
© 2019 SPLUNK INC.
Pulsar Functions
● Functions can store state in stream storage
● State is global and replicated
● Multiple instances of the same function can access the same state
● Functions framework provides simple abstraction over state
State management
© 2019 SPLUNK INC.
Pulsar Functions
● Implemented on top of Apache BookKeeper “Table Service”
● BookKeeper provides a sharded key/value store based on:
○ Log & Snapshot - Stored as BookKeeper ledgers
○ Warm replicas that can be quickly promoted to leader
● In case of leader failure there is no downtime or huge log to replay
State management
© 2019 SPLUNK INC.
Pulsar Functions
State example
import org.apache.pulsar.functions.api.Context;
import org.apache.pulsar.functions.api.PulsarFunction;
public class CounterFunction
implements PulsarFunction<String, Void> {
@Override
public Void process(String input, Context context) {
for (String word : input.split(".")) {
context.incrCounter(word, 1);
}
return null;
}
}
© 2019 SPLUNK INC.
Pulsar IO
Connectors Framework based on Pulsar Functions
© 2019 SPLUNK INC.
Built-in Pulsar IO connectors
© 2019 SPLUNK INC.
Querying data stored
in Pulsar
© 2019 SPLUNK INC.
Pulsar SQL
● Uses Presto for interactive SQL
queries over data stored in Pulsar
● Query historic and real-time data
● Integrated with schema registry
● Can join with data from other
sources
© 2019 SPLUNK INC.
Pulsar SQL
● Read data directly from BookKeeper into Presto — bypass Pulsar Broker
● Many-to-many data reads
○ Data is split even on a single partition — multiple workers can read data in parallel from single
Pulsar partition
● Time based indexing — Use “publishTime” in predicates to reduce data being read
from disk
© 2019 SPLUNK INC.
Pulsar Storage API
● Work in progress to allow direct access to data stored in Pulsar
● Generalization of the work done for Presto connector
● Most efficient way to retrieve and process data from “batch” execution engines
Thank You
© 2019 SPLUNK INC.

Weitere ähnliche Inhalte

Was ist angesagt?

Serverless Streaming Architectures and Algorithms for the Enterprise
Serverless Streaming Architectures and Algorithms for the EnterpriseServerless Streaming Architectures and Algorithms for the Enterprise
Serverless Streaming Architectures and Algorithms for the Enterprise
Arun Kejariwal
 

Was ist angesagt? (20)

Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
 
Migrating Airflow-based Apache Spark Jobs to Kubernetes – the Native Way
Migrating Airflow-based Apache Spark Jobs to Kubernetes – the Native WayMigrating Airflow-based Apache Spark Jobs to Kubernetes – the Native Way
Migrating Airflow-based Apache Spark Jobs to Kubernetes – the Native Way
 
Effectively-once semantics in Apache Pulsar
Effectively-once semantics in Apache PulsarEffectively-once semantics in Apache Pulsar
Effectively-once semantics in Apache Pulsar
 
Scaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/dayScaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/day
 
Building Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeBuilding Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta Lake
 
Modern real-time streaming architectures
Modern real-time streaming architecturesModern real-time streaming architectures
Modern real-time streaming architectures
 
Deploying Flink on Kubernetes - David Anderson
 Deploying Flink on Kubernetes - David Anderson Deploying Flink on Kubernetes - David Anderson
Deploying Flink on Kubernetes - David Anderson
 
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DBDistributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
 
Using ClickHouse for Experimentation
Using ClickHouse for ExperimentationUsing ClickHouse for Experimentation
Using ClickHouse for Experimentation
 
Presto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performancePresto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performance
 
Cloud Native PostgreSQL
Cloud Native PostgreSQLCloud Native PostgreSQL
Cloud Native PostgreSQL
 
Better than you think: Handling JSON data in ClickHouse
Better than you think: Handling JSON data in ClickHouseBetter than you think: Handling JSON data in ClickHouse
Better than you think: Handling JSON data in ClickHouse
 
Pulsar in the Lakehouse: Apache Pulsar™ with Apache Spark™ and Delta Lake - P...
Pulsar in the Lakehouse: Apache Pulsar™ with Apache Spark™ and Delta Lake - P...Pulsar in the Lakehouse: Apache Pulsar™ with Apache Spark™ and Delta Lake - P...
Pulsar in the Lakehouse: Apache Pulsar™ with Apache Spark™ and Delta Lake - P...
 
Ingesting data at scale into elasticsearch with apache pulsar
Ingesting data at scale into elasticsearch with apache pulsarIngesting data at scale into elasticsearch with apache pulsar
Ingesting data at scale into elasticsearch with apache pulsar
 
Serverless Streaming Architectures and Algorithms for the Enterprise
Serverless Streaming Architectures and Algorithms for the EnterpriseServerless Streaming Architectures and Algorithms for the Enterprise
Serverless Streaming Architectures and Algorithms for the Enterprise
 
How Orange Financial combat financial frauds over 50M transactions a day usin...
How Orange Financial combat financial frauds over 50M transactions a day usin...How Orange Financial combat financial frauds over 50M transactions a day usin...
How Orange Financial combat financial frauds over 50M transactions a day usin...
 
Redpanda and ClickHouse
Redpanda and ClickHouseRedpanda and ClickHouse
Redpanda and ClickHouse
 
Introducing Apache Airflow and how we are using it
Introducing Apache Airflow and how we are using itIntroducing Apache Airflow and how we are using it
Introducing Apache Airflow and how we are using it
 
KFServing and Feast
KFServing and FeastKFServing and Feast
KFServing and Feast
 
A Day in the Life of a ClickHouse Query Webinar Slides
A Day in the Life of a ClickHouse Query Webinar Slides A Day in the Life of a ClickHouse Query Webinar Slides
A Day in the Life of a ClickHouse Query Webinar Slides
 

Ähnlich wie Apache Pulsar: The Next Generation Messaging and Queuing System

Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
Timothy Spann
 
JConWorld_ Continuous SQL with Kafka and Flink
JConWorld_ Continuous SQL with Kafka and FlinkJConWorld_ Continuous SQL with Kafka and Flink
JConWorld_ Continuous SQL with Kafka and Flink
Timothy Spann
 

Ähnlich wie Apache Pulsar: The Next Generation Messaging and Queuing System (20)

Interactive querying of streams using Apache Pulsar_Jerry peng
Interactive querying of streams using Apache Pulsar_Jerry pengInteractive querying of streams using Apache Pulsar_Jerry peng
Interactive querying of streams using Apache Pulsar_Jerry peng
 
Scaling Apache Pulsar to 10 Petabytes/Day - Pulsar Summit NA 2021 Keynote
Scaling Apache Pulsar to 10 Petabytes/Day - Pulsar Summit NA 2021 KeynoteScaling Apache Pulsar to 10 Petabytes/Day - Pulsar Summit NA 2021 Keynote
Scaling Apache Pulsar to 10 Petabytes/Day - Pulsar Summit NA 2021 Keynote
 
Apache Pulsar @Splunk
Apache Pulsar @SplunkApache Pulsar @Splunk
Apache Pulsar @Splunk
 
Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2 Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2
 
Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2 Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2
 
Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2Splunk Cloud and Splunk Enterprise 7.2
Splunk Cloud and Splunk Enterprise 7.2
 
What's New with the Latest Splunk Platform Release
What's New with the Latest Splunk Platform ReleaseWhat's New with the Latest Splunk Platform Release
What's New with the Latest Splunk Platform Release
 
Pulsar summit-keynote-final
Pulsar summit-keynote-finalPulsar summit-keynote-final
Pulsar summit-keynote-final
 
Alle Neuigkeiten im letzten Plattform Release
Alle Neuigkeiten im letzten Plattform ReleaseAlle Neuigkeiten im letzten Plattform Release
Alle Neuigkeiten im letzten Plattform Release
 
Apache Kafka
Apache Kafka Apache Kafka
Apache Kafka
 
Powering Data Science and AI with Apache Spark, Alluxio, and IBM
Powering Data Science and AI with Apache Spark, Alluxio, and IBMPowering Data Science and AI with Apache Spark, Alluxio, and IBM
Powering Data Science and AI with Apache Spark, Alluxio, and IBM
 
iovlabs.pdf
iovlabs.pdfiovlabs.pdf
iovlabs.pdf
 
Kubernetes Forum Seoul 2019: Re-architecting Data Platform with Kubernetes
Kubernetes Forum Seoul 2019: Re-architecting Data Platform with KubernetesKubernetes Forum Seoul 2019: Re-architecting Data Platform with Kubernetes
Kubernetes Forum Seoul 2019: Re-architecting Data Platform with Kubernetes
 
Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
Budapest Data/ML - Building Modern Data Streaming Apps with NiFi, Flink and K...
 
Event driven architectures with Kinesis
Event driven architectures with KinesisEvent driven architectures with Kinesis
Event driven architectures with Kinesis
 
AWS re:Invent 2016 - Scality's Open Source AWS S3 Server
AWS re:Invent 2016 - Scality's Open Source AWS S3 ServerAWS re:Invent 2016 - Scality's Open Source AWS S3 Server
AWS re:Invent 2016 - Scality's Open Source AWS S3 Server
 
Disaggregation a Primer: Optimizing design for Edge Cloud & Bare Metal applic...
Disaggregation a Primer: Optimizing design for Edge Cloud & Bare Metal applic...Disaggregation a Primer: Optimizing design for Edge Cloud & Bare Metal applic...
Disaggregation a Primer: Optimizing design for Edge Cloud & Bare Metal applic...
 
PyCon HK 2018 - Heterogeneous job processing with Apache Kafka
PyCon HK 2018 - Heterogeneous job processing with Apache Kafka PyCon HK 2018 - Heterogeneous job processing with Apache Kafka
PyCon HK 2018 - Heterogeneous job processing with Apache Kafka
 
Strimzi - Where Apache Kafka meets OpenShift - OpenShift Spain MeetUp
Strimzi - Where Apache Kafka meets OpenShift - OpenShift Spain MeetUpStrimzi - Where Apache Kafka meets OpenShift - OpenShift Spain MeetUp
Strimzi - Where Apache Kafka meets OpenShift - OpenShift Spain MeetUp
 
JConWorld_ Continuous SQL with Kafka and Flink
JConWorld_ Continuous SQL with Kafka and FlinkJConWorld_ Continuous SQL with Kafka and Flink
JConWorld_ Continuous SQL with Kafka and Flink
 

Mehr von Databricks

Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
Databricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
Databricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Databricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
Databricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
Databricks
 

Mehr von Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on Kubernetes
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 

Kürzlich hochgeladen

Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
amitlee9823
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
amitlee9823
 
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
MarinCaroMartnezBerg
 
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
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
amitlee9823
 
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
amitlee9823
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
amitlee9823
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
amitlee9823
 
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts ServiceCall Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
ZurliaSoop
 
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
amitlee9823
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
AroojKhan71
 

Kürzlich hochgeladen (20)

Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
 
Sampling (random) method and Non random.ppt
Sampling (random) method and Non random.pptSampling (random) method and Non random.ppt
Sampling (random) method and Non random.ppt
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
Predicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science ProjectPredicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science Project
 
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
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
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
 
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...
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
 
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptx
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
 
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts ServiceCall Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
 
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
 
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
 

Apache Pulsar: The Next Generation Messaging and Queuing System

  • 1. © 2019 SPLUNK INC. The Next Generation Messaging and Queuing System
  • 2. © 2019 SPLUNK INC. Intro Senior Principal Engineer - Splunk Co-creator Apache Pulsar Matteo Merli Senior Director of Engineering - Splunk Karthik Ramasamy
  • 3. © 2019 SPLUNK INC. Messaging and Streaming
  • 4. © 2019 SPLUNK INC. Messaging Message passing between components, application, services
  • 5. © 2019 SPLUNK INC. Streaming Analyze events that just happened
  • 6. © 2019 SPLUNK INC. Messaging vs Streaming 2 worlds, 1 infra
  • 7. © 2019 SPLUNK INC. Use cases ● OLTP, Integration ● Main challenges: ○ Latency ○ Availability ○ Data durability ○ High level features ■ Routing, DLQ, delays, individual acks ● Real-time analytics ● Main challenges: ○ Throughput ○ Ordering ○ Stateful processing ○ Batch + Real-Time Messaging Streaming
  • 8. © 2019 SPLUNK INC. Storage Messaging Compute
  • 9. © 2019 SPLUNK INC. Apache Pulsar Data replicated and synced to disk Durability Low publish latency of 5ms at 99pct Low Latency Can reach 1.8 M messages/s in a single partition High Throughput System is available if any 2 nodes are up High Availability Take advantage of dynamic cluster scaling in cloud environments Cloud Native Flexible Pub-Sub and Compute backed by durable log storage
  • 10. © 2019 SPLUNK INC. Apache Pulsar Support both Topic & Queue semantic in a single model Unified messaging model Can support millions of topics Highly Scalable Lightweight compute framework based on functions Native Compute Supports multiple users and workloads in a single cluster Multi Tenant Out of box support for geographically distributed applications Geo Replication Flexible Pub-Sub and Compute backed by durable log storage
  • 11. © 2019 SPLUNK INC. Apache Pulsar project in numbers 192 Contributors 30 Committers 100s Adopters 4.6K Github Stars
  • 12. © 2019 SPLUNK INC. Sample of Pulsar users and contributors
  • 13. © 2019 SPLUNK INC. Messaging Model
  • 14. © 2019 SPLUNK INC. Pulsar Client libraries ● Java — C++ — C — Python — Go — NodeJS — WebSocket APIs ● Partitioned topics ● Apache Kafka compatibility wrapper API ● Transparent batching and compression ● TLS encryption and authentication ● End-to-end encryption
  • 15. © 2019 SPLUNK INC. Architectural view Separate layers between brokers bookies ● Broker and bookies can be added independently ● Traffic can be shifted very quickly across brokers ● New bookies will ramp up on traffic quickly
  • 16. © 2019 SPLUNK INC. Apache BookKeeper ● Low-latency durable writes ● Simple repeatable read consistency ● Highly available ● Store many logs per node ● I/O Isolation Replicated log storage
  • 17. © 2019 SPLUNK INC. Inside BookKeeper Storage optimized for sequential & immutable data ● IO isolation between write and read operations ● Does not rely on OS page cache ● Slow consumers won’t impact latency ● Very effective IO patterns: ○ Journal — append only and no reads ○ Storage device — bulk write and sequential reads ● Number of files is independent from number of topics
  • 18. © 2019 SPLUNK INC. Segment Centric Storage In addition to partitioning, messages are stored in segments (based on time and size) Segments are independent from each others and spread across all storage nodes
  • 19. © 2019 SPLUNK INC. Segments vs Partitions
  • 20. © 2019 SPLUNK INC. Tiered Storage Unlimited topic storage capacity Achieves the true “stream-storage”: keep the raw data forever in stream form Extremely cost effective
  • 21. © 2019 SPLUNK INC. Schema Registry Store information on the data structure — Stored in BookKeeper Enforce data types on topic Allow for compatible schema evolutions
  • 22. © 2019 SPLUNK INC. Schema Registry ● Integrated schema in API ● End-to-end type safety — Enforced in Pulsar broker Producer<MyClass> producer = client .newProducer(Schema.JSON(MyClass.class)) .topic("my-topic") .create(); producer.send(new MyClass(1, 2)); Consumer<MyClass> consumer = client .newConsumer(Schema.JSON(MyClass.class)) .topic("my-topic") .subscriptionName("my-subscription") .subscribe(); Message<MyClass> msg = consumer.receive(); Type Safe API
  • 23. © 2019 SPLUNK INC. Geo Replication Scalable asynchronous replication Integrated in the broker message flow Simple configuration to add/remove regions
  • 24. © 2019 SPLUNK INC. Replicated Subscriptions ● Consumption will restart close to where a consumer left off - Small amount of dups ● Implementation ○ Use markers injected into the data flow ○ Create a consistent snapshot of message ids across cluster ○ Establish a relationship: If consumed MA-1 in Cluster-A it must have consumed MB-2 in Cluster-B Migrate subscriptions across geo-replicated clusters
  • 25. © 2019 SPLUNK INC. Multi-Tenancy ● Authentication / Authorization / Namespaces / Admin APIs ● I/O Isolations between writes and reads ○ Provided by BookKeeper ○ Ensure readers draining backlog won’t affect publishers ● Soft isolation ○ Storage quotas — flow-control — back-pressure — rate limiting ● Hardware isolation ○ Constrain some tenants on a subset of brokers or bookies A single Pulsar cluster supports multiple users and mixed workloads
  • 26. © 2019 SPLUNK INC. Lightweight Compute with Pulsar Functions
  • 27. © 2019 SPLUNK INC. Pulsar Functions
  • 28. © 2019 SPLUNK INC. Pulsar Functions ● User supplied compute against a consumed message ○ ETL, data enrichment, filtering, routing ● Simplest possible API ○ Use language specific “function” notation ○ No SDK required ○ SDK available for more advanced features (state, metrics, logging, …) ● Language agnostic ○ Java, Python and Go ○ Easy to support more languages ● Pluggable runtime ○ Managed or manual deployment ○ Run as threads, processes or containers in Kubernetes
  • 29. © 2019 SPLUNK INC. Pulsar Functions def process(input): return input + '!' import java.util.function.Function; public class ExclamationFunction implements Function<String, String> { @Override public String apply(String input) { return input + "!"; } } Python Java Examples
  • 30. © 2019 SPLUNK INC. Pulsar Functions ● Functions can store state in stream storage ● State is global and replicated ● Multiple instances of the same function can access the same state ● Functions framework provides simple abstraction over state State management
  • 31. © 2019 SPLUNK INC. Pulsar Functions ● Implemented on top of Apache BookKeeper “Table Service” ● BookKeeper provides a sharded key/value store based on: ○ Log & Snapshot - Stored as BookKeeper ledgers ○ Warm replicas that can be quickly promoted to leader ● In case of leader failure there is no downtime or huge log to replay State management
  • 32. © 2019 SPLUNK INC. Pulsar Functions State example import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.PulsarFunction; public class CounterFunction implements PulsarFunction<String, Void> { @Override public Void process(String input, Context context) { for (String word : input.split(".")) { context.incrCounter(word, 1); } return null; } }
  • 33. © 2019 SPLUNK INC. Pulsar IO Connectors Framework based on Pulsar Functions
  • 34. © 2019 SPLUNK INC. Built-in Pulsar IO connectors
  • 35. © 2019 SPLUNK INC. Querying data stored in Pulsar
  • 36. © 2019 SPLUNK INC. Pulsar SQL ● Uses Presto for interactive SQL queries over data stored in Pulsar ● Query historic and real-time data ● Integrated with schema registry ● Can join with data from other sources
  • 37. © 2019 SPLUNK INC. Pulsar SQL ● Read data directly from BookKeeper into Presto — bypass Pulsar Broker ● Many-to-many data reads ○ Data is split even on a single partition — multiple workers can read data in parallel from single Pulsar partition ● Time based indexing — Use “publishTime” in predicates to reduce data being read from disk
  • 38. © 2019 SPLUNK INC. Pulsar Storage API ● Work in progress to allow direct access to data stored in Pulsar ● Generalization of the work done for Presto connector ● Most efficient way to retrieve and process data from “batch” execution engines
  • 39. Thank You © 2019 SPLUNK INC.