SlideShare ist ein Scribd-Unternehmen logo
1 von 50
Design and
Implementation of Spark
Streaming Connectors
ARIJIT TARAFDAR
NAN ZHU
MARCH 23,2017
About Us
Arijit Tarafdar
Software Engineer@Azure HDInsight. Work on Spark Streaming/Structured Streaming
service in Azure. Committee Member of XGBoost@DMLC and Apache MxNet (incubator).
Spark Contributor. Known as CodingCat in GitHub.
Nan Zhu
Software Engineer@Azure HDInsight. Work on Spark/Spark Streaming on Azure.
Previously worked with other distributed platforms like DryadLinq and MPI. Also
worked on graph coloring algorithms which was contributed to ADOL-C
(https://projects.coin-or.org/ADOL-C).
Real Time Data Analytics Results
Processing Engine
Continuous Data Source Control Manager
Continuous Data Source API
Persistent Data Storage Layer
Spark Streaming, Structured Streaming
Deliver real time data to Spark at scale
Real time view of data (message queue
or files filtered by timestamp)
Blobs/Queues/Tables/Files
Continuous Application Architecture and Role of Spark Connectors
Not only size of data is increasing, but also the velocity of data
◦ Sensors, IoT devices, social networks and online transactions are all generating
data that needs to be monitored constantly and acted upon quickly.
Outline
•Recap of Spark Streaming
•Introduction to Event Hubs
•Connecting Azure Event Hubs and Spark Streaming
•Design Considerations for Spark Streaming Connector
•Contributions Back to Community
•Future Work
Spark Streaming - Background
Task 1
Task 2
Task L
RDD 1 @ t RDD 1 @ t-1 RDD 1 @ 0
Stream 1
Task 1
Task 2
Task M
RDD N @ t RDD N @ t-1 RDD N @ 0
Stream N
Micro Batch @ t
Task 1
Task 2
Task L
Task 1
Task 2
Task M
Window Duration
Batch Duration
Azure Event Hubs - Introduction
Partition 1
Partition 2
Partition J
Event Hubs 1
Partition 1
Partition 2
Partition K
Event Hubs L
Event Hubs Namespace 1
Partition 1
Partition 2
Partition K
Event Hubs 1
Partition 1
Partition 2
Partition P
Event Hubs N
Event Hubs Namespace M
Azure Event Hubs - Introduction
https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-what-is-event-hubs
Data Flow – Event Hubs
• Proactive message delivery
• Efficient in terms of communication cost
• Data source treated as commit log of events
• Events read in batch per receive() call
New Old
Event Hubs Partition
(Event Hubs Server)
Prefetch Queue
(Event Hubs Client)
Streaming
Application
Event Hubs – Offset Management
• Event Hubs expect offset management to be performed on the receiver side
• Spark streaming uses DFS based persistent store (HDFS, ADLS, etc.)
• Offset is stored per consumer group per partition per event hubs per event hubs namespace
/* An interface to read/write offset for a given Event Hubs
namespace/name/partition */
@SerialVersionUID(1L)
trait OffsetStore extends Serializable {
def open(): Unit
def write(offset: String): Unit
def read() : String
def close(): Unit
}
First Version: Receiver-based
Spark Streaming Connector for
Azure Event Hub
Application
Driver (Spark)
Receiver
Executor (Spark)
Streaming
Context
Spark
Context
Eventhubs
Receiver
Task Executor
(Spark)
User Defined
Functions
ADLS
ADLS
Write Ahead Log (WAL)
Checkpoint Directory
Memory
Block
Data
Block
Metadata
Jobs
Tasks
Checkpoint
Data
Azure
Eventhubs
Input
Stream
ADLS
WASB
Output Storage
Fault Tolerance – Spark Receiver Based Event Hubs Connector
Restarted
Application
Driver (Spark)
Restarted Receiver
Executor (Spark)
Restarted
Streaming
Context
Restarted
Spark
Context
Restarted
Eventhubs
Receiver
Restarted Task
Executor (Spark)
User Defined
Functions
ADLS
ADLS
Write Ahead Log (WAL)
Checkpoint Directory
Memory
Recover Block
Data
Recover Block
Metadata
Jobs
Tasks
Restart
Computation
From
Checkpoint
Data Azure
Eventhubs
ADLS
WASB
Output Storage
Spark Streaming – Recovery After Failure
Event Hubs Receiver – Class Signature
private[eventhubs] class EventHubsReceiver(
eventhubsParams: Map[String, String],
partitionId: String,
storageLevel: StorageLevel,
offsetStore: Option[OffsetStore],
receiverClient: EventHubsClientWrapper,
maximumEventRate: Int) extends
Receiver[Array[Byte]](storageLevel) with
Logging { ... }
Event Hubs Receiver – Methods Used/Implemented
@DeveloperApi
abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable {
def onStart(): Unit
def onStop(): Unit
def store(dataItem: T) {
supervisor.pushSingle(dataItem)
}
def store(dataBuffer: ArrayBuffer[T]) {
supervisor.pushArrayBuffer(dataBuffer, None, None)
}
def restart(message: String, error: Throwable) {
supervisor.restartReceiver(message, Some(error))
}
}
Azure Event Hubs/Spark
Expectations
Receiver-based Connection
(Event Hubs) Long-Running
Receiver/Proactive Message
Fetching
Long-Running Receiver Tasks
(Spark) Logging Data Before
Ack
WAL/Spark Checkpoint
(Event Hubs) Client-side
Offset Management
Offset Store
A Natural Fit!!!
Why Receiver based connector?
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Lessons learnt from based connector?
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Logging Data
Before Ack
WAL/Spark
Checkpoint
Performance/Data
loss due to Spark
bug/No easy
recovery from code
update
https://issues.apache.org/jira/browse/SPARK-18957
Lessons learnt from based connector?
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Logging Data
Before Ack
WAL/Spark
Checkpoint
Performance/Data
loss due to Spark
bug/No easy
recovery from code
update
Client-side Offset
Management
Offset Store Looks fine….
Lessons learnt from based connector?
Bridging Spark Streaming and
Event Hubs WITHOUT Receiver
How the Idea Extends to Other
Data Sources (in Azure & Your IT
Infrastructure)?
Extra Resources
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Fault tolerance
Mechanism
WAL/Spark
Checkpoint
Perf./Data Loss due
to Spark Bug/No
Recovery from
Code Update
Client-side Offset
Management
Offset Store Looks fine….
From Event Hubs to General
Data Sources (1)
•Communication Pattern
• Azure Event Hubs: Long-Running Receiver, Proactive Data Delivery
• Kafka: Receiver Start/Shutdown in a free-style, Passive Data
Delivery
Most Critical Factor in Designing a Resource-Efficient
Spark Streaming Connector!
Tackling Extra Resource
Requirement
Azure EventHubs
EvH-
Namespace-1
EventHub-1
P1
PN
.
.
.
Reduce Resource Requirements:
Customized Receiver
Logic
User-Defined
Lambdas
EventHubsRDD
.map()
MapPartitionsRDD
Spark Tasks
Compact Data Receiving and Processing in the same Task
Inspired by Kafka
Direct DStream!
Being More Challenging with a
Different Communication Pattern!
Bridging Spark Execution Model and
Communication Pattern Expectation
Azure EventHubs
EvH-
Namespace-1
EventHub-1
P1
PN
.
.
.
Customized Receiver
Logic
User-Defined
Lambdas
EventHubsRDD
.map()
MapPartitionsRDD
Spark Task
Passive
Message
Delivery Layer
Recv(expectedMsgNum:
Int) – Blocking API
Long-running/Proactive Receiver expected by Event Hubs
vs.
Transient Tasks started for each Batch by Spark
Takeaways (1)
Requirements in
Event Hubs
Receiver-based
Connection
Problems Solution
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Compact Data
Receiving/Processi
ng, with the
facilitates from
Passive Message
Delivery
Communication Pattern in Data Sources Plays the Key
Role in Resource-Efficient Design of Spark Streaming
Connector
Next Problem…
Fault Tolerance
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Fault tolerance
Mechanism
WAL/Spark
Checkpoint
Perf./Data Loss due
to Spark Bug/No
Recovery from Code
Update
Client-side Offset
Management
Offset Store Looks fine….
From Event Hubs to General
Data Sources (2)
•Fault-Tolerance
• Capability
• Guarantee graceful recovery (no data loss, recover from where
you stopped, etc.) with application stops for various reasons
• Efficiency
• Minimum impact to application performance and user
deployment
…RDD L-t RDD L-(t-1) RDD L-0 Stream L
Unexpected Application Stop
Checkpoint Time
RDD L-(t-1)RDD L-t
Recovery
From Checkpoint, or Re-evaluated
Capability – Recover from
unexpected stop
…RDD L-(t-1) RDD L-0 Stream L
Application Upgrade
…
Application Stop
Spark Checkpoint Mechanism Serializes
Everything and does not recognize a re-compiled
class
Capability – Recover from
planned stop
RDD L-(2t)
Resume Application
with updated
Implementation
Fetch the latest offset
Offset Store
Your Connector shall maintain this!!!
Efficiency - What to be
Contained in Checkpoint Files?
• Checkpointing takes your computing resources!!!
• Received Event Data
• too large
• The range of messages to be processed in each batch
• Small enough to quickly persist data
Azure EventHubs
EvH-
Namespace-1
EventHub-1
P1
PN
.
.
.
EventHubsRDD
.map()
MapPartitionsRDD
Passive
Message
Delivery Layer
Recv(expectedMsgNum:
Int) – Blocking API
Persist this mapping relationship, i.e. using EventHubs itself as data backup
Efficiency - Checkpoint
Cleanup
•Connectors for Data Source Requiring Client-
side offset management generates Data/Files
for each Batch
• You have to clean up SAFELY
• Keep recovery feasible
• Coordinate with Spark’s checkpoint process
• Override clearCheckpointData() in EventHubsDStream (our
implementation of Dstream)
• Triggered by Batch Completion
• Delete all offset records out of the remembering window
Takeaways (2)
Requirements in
Event Hubs
Receiver-based
Connection
Problems Solution
Fault tolerance
Mechanism
WAL/Spark
Checkpoint
Perf./Data Loss due
to Spark Bug/No
Recovery from Code
Update
Checkpoint Mapping
Relationship instead
of Data/Self-
management Offset
Store/Coordinate
Checkpoint Cleanup
Fault Tolerance Design is about Interaction with Spark
Streaming Checkpoint
No Problem any more?
Offset Management
Requirements in
Event Hubs
Receiver-based
Connection
Problems
Long-Running
Receiver/Proactive
Message Fetching
Long-Running
Receiver Tasks
Extra Resource
Requirements
Fault tolerance
Mechanism
WAL/Spark
Checkpoint
Data Loss due to
Spark Bug
Client-side Offset
Management
Offset Store Looks fine….
Is it really fine???
From Event Hubs to General
Data Sources (3)
•Message Addressing Rate Control
Message Addressing
• Why Message Addressing?
• When creating a client instance of data source in a Spark task, where to start receiving?
• Without this info, you have to replay the stream for every newly created client
Data
Source
Client
Start from the first msg
Data
Source
Client
Start from where?
• Design Options:
• Xth message (X: 0, 1, 2, 3, 4….)
• server side metadata to map the message ID to the offset in storage system
• Actual offset
• Simpler server side design
Fault
Or
Next Batch
Rate Control
• Why Rate Control
• Prevent the messages flooding into the processing pipelines
• e.g. just start processing a queued up data sources
• Design Options
• Number of messages: I want to consume 1000 messages in next batch
• Assuming the homogeneous processing overhead
• Size of messages: I want to receive at most 1000 bytes in next batch
• Complicated Server side logic -> track the delivered size
• Larger messages, longer processing time is not always true
Data
Source
Client
Start from the first msg
Data
Source
Client
Consume all messages at
once? May crash your
processing engine!!!A Long Stop!!!
Kafka Choice
• Message Addressing:
• Xth message: 0, 1, 2, 3, 4, ..
• Rate Control
• Number of Messages: 0, 1, 2, 3, 4, …
Driver
Executor
Executor
Kafka
Message Addressing and Rate Control:
Batch 0: How many messages are to be
processed in next batch, and where to start? 0
- 999
Batch 1: How many messages are to be processed
in next batch, and where to start? 1000 - 1999
Azure Event Hubs’ Choice
• Message Addressing:
• Offset of messages: 0, size of msg 0, size of (msg 0 + msg 1),…
• Rate Control
• Number of Messages: 0, 1, 2, 3, 4, …
This brings totally different connector
design/implementation!!!
Distributed Information for Rate
Control and Message Addressing
Driver
Executor
Executor
Rate Control:
Batch 0: How many messages are to
be processed in next batch, and
where to start? 0 - 999
Azure EventHubs
Batch 1: How many messages are to be
processed in next batch, and where to
start? 1000 - 1999
What’s the offset of
1000th message???
The answer appeared in Executor
side (when Task receives the
message (as part of message
metadata))
Build a Channel to Pass
Information from Executor to
Driver!!!
HDFS-based Channel
Implementation
LastOffset_P1_Batch_i
LastOffset_PN_Batch_i
EventHubsRDD
Tasks
.map()
MapPartitionsRDD
Tasks
What’s the next step??? Simply let
Driver-side logic read the files?
•APIs like RDD.take(x) evaluates only some of the partitions…Batch 0
generate 3 files, and Batch 1 generates 5 files…
•You have to merge the latest files with the historical results and
commit and then direct the driver-side logic to read
No!!!
HDFS-based Channel
Implementation
LastOffset_P1_Batch_i
LastOffset_PN_Batch_i
EventHubsRDD
Tasks
.map()
MapPartitionsRDD
Tasks
•APIs like RDD.take(x) evaluates only some of the partitions…Batch 0
generate 3 files, and Batch 1 generates 5 files…
•You have to merge the latest files with the historical results and
commit ...
•Ensure that all streams’ offset are committed transactionally
•Discard the partially merged/committed results to rerun the batch
HDFS-based Channel
Implementation
RDD Generation “Thread” Job Execution “Thread”
Generate RDD
Blocking
(wait)
Processing
RDD
BatchComplete
Event
SparkListenerBus
CustomizedListen
er
CommitOffsets
and Notify
HDFS-based Channel
Implementation
RDD Generation “Thread” Job Execution “Thread”
Generate RDD
Blocking
(wait)
Processing
Micro Batch
BatchComplete
Event
SparkListenerBus
CustomizedListen
er
CommitOffsets
and Notify
DStream.graph: DStreamGraph
Takeaways (3)
• There are multiple options on the Server-side design for Message
Addressing and Rate Control
• To design and implement a Spark Streaming connector, you have to
understand what are the options adopted in server side
The key is the combination!!!
Contribute Back to
Community
Failed Recovery from checkpoint caused by the multi-threads issue in
Spark Streaming scheduler
https://issues.apache.org/jira/browse/SPARK-19280
One Realistic Example of its Impact: You are potentially getting wrong
data when you use Kafka and reduceByWindow and recover from a
failure
Data loss caused by improper post-batch-completed processing
https://issues.apache.org/jira/browse/SPARK-18905
Inconsistent Behavior of Spark Streaming Checkpoint
https://issues.apache.org/jira/browse/SPARK-19233
Summary
• Spark Streaming Connector for Azure Event Hubs enables the user to perform
various types of analytics over streaming data from a fully managed, cloud-scale
message telemetry ingestion service
• https://github.com/hdinsight/spark-eventhubs
• Design and Implementation of Spark Streaming Connectors
• Coordinate Execution Model and Communication Pattern
• Fault Tolerance (Spark Streaming Checkpoint v.s. self-managed fault tolerance facilitates)
• Message Addressing and Rate Control (Server&Connector Co-Design)
• Contributing Back to the Community
• Microsoft is the organization with the most open source contributors in 2016!!!
• http://www.businessinsider.com/microsoft-github-open-source-2016-9
If you do not want to handle
this complexity
Move to Azure HDInsight…
Future Work
Structured Streaming integration with Event Hubs (will release at the
end of month)
Streaming Data Visualization with PowerBI (alpha released mode)
Streaming ETL Solutions on Azure HDInsight!
Thank You!!!
Build a Powerful&Robust Data Analytic
Pipeline with Spark@Azure HDInsight!!!

Weitere ähnliche Inhalte

Was ist angesagt?

Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Timothy Spann
 
Using the flipn stack for edge ai (flink, nifi, pulsar)
Using the flipn stack for edge ai (flink, nifi, pulsar)Using the flipn stack for edge ai (flink, nifi, pulsar)
Using the flipn stack for edge ai (flink, nifi, pulsar)Timothy Spann
 
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...Timothy Spann
 
ApacheCon 2021: Apache NiFi 101- introduction and best practices
ApacheCon 2021:   Apache NiFi 101- introduction and best practicesApacheCon 2021:   Apache NiFi 101- introduction and best practices
ApacheCon 2021: Apache NiFi 101- introduction and best practicesTimothy Spann
 
Music city data Hail Hydrate! from stream to lake
Music city data Hail Hydrate! from stream to lakeMusic city data Hail Hydrate! from stream to lake
Music city data Hail Hydrate! from stream to lakeTimothy Spann
 
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
Pulsar summit asia 2021   apache pulsar with mqtt for edge computingPulsar summit asia 2021   apache pulsar with mqtt for edge computing
Pulsar summit asia 2021 apache pulsar with mqtt for edge computingTimothy Spann
 
Spark summit-east-dowling-feb2017-full
Spark summit-east-dowling-feb2017-fullSpark summit-east-dowling-feb2017-full
Spark summit-east-dowling-feb2017-fullJim Dowling
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkDataWorks Summit
 
DBCC 2021 - FLiP Stack for Cloud Data Lakes
DBCC 2021 - FLiP Stack for Cloud Data LakesDBCC 2021 - FLiP Stack for Cloud Data Lakes
DBCC 2021 - FLiP Stack for Cloud Data LakesTimothy Spann
 
Osacon 2021 hello hydrate! from stream to clickhouse with apache pulsar and...
Osacon 2021   hello hydrate! from stream to clickhouse with apache pulsar and...Osacon 2021   hello hydrate! from stream to clickhouse with apache pulsar and...
Osacon 2021 hello hydrate! from stream to clickhouse with apache pulsar and...Timothy Spann
 
Codeless pipelines with pulsar and flink
Codeless pipelines with pulsar and flinkCodeless pipelines with pulsar and flink
Codeless pipelines with pulsar and flinkTimothy Spann
 
Interactive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using DruidInteractive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using DruidDataWorks Summit/Hadoop Summit
 
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...DataWorks Summit
 
fluentd -- the missing log collector
fluentd -- the missing log collectorfluentd -- the missing log collector
fluentd -- the missing log collectorMuga Nishizawa
 
Matt Franklin - Apache Software (Geekfest)
Matt Franklin - Apache Software (Geekfest)Matt Franklin - Apache Software (Geekfest)
Matt Franklin - Apache Software (Geekfest)W2O Group
 
Overview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics FrameworksOverview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics FrameworksDataWorks Summit/Hadoop Summit
 
ApacheCon 2021: Cracking the nut with Apache Pulsar (FLiP)
ApacheCon 2021:  Cracking the nut with Apache Pulsar (FLiP)ApacheCon 2021:  Cracking the nut with Apache Pulsar (FLiP)
ApacheCon 2021: Cracking the nut with Apache Pulsar (FLiP)Timothy Spann
 
Cloud lunch and learn real-time streaming in azure
Cloud lunch and learn real-time streaming in azureCloud lunch and learn real-time streaming in azure
Cloud lunch and learn real-time streaming in azureTimothy Spann
 
Scenic City Summit (2021): Real-Time Streaming in any and all clouds, hybrid...
Scenic City Summit (2021):  Real-Time Streaming in any and all clouds, hybrid...Scenic City Summit (2021):  Real-Time Streaming in any and all clouds, hybrid...
Scenic City Summit (2021): Real-Time Streaming in any and all clouds, hybrid...Timothy Spann
 

Was ist angesagt? (20)

Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4
 
Using the flipn stack for edge ai (flink, nifi, pulsar)
Using the flipn stack for edge ai (flink, nifi, pulsar)Using the flipn stack for edge ai (flink, nifi, pulsar)
Using the flipn stack for edge ai (flink, nifi, pulsar)
 
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...
Live Demo Jam Expands: The Leading-Edge Streaming Data Platform with NiFi, Ka...
 
ApacheCon 2021: Apache NiFi 101- introduction and best practices
ApacheCon 2021:   Apache NiFi 101- introduction and best practicesApacheCon 2021:   Apache NiFi 101- introduction and best practices
ApacheCon 2021: Apache NiFi 101- introduction and best practices
 
Music city data Hail Hydrate! from stream to lake
Music city data Hail Hydrate! from stream to lakeMusic city data Hail Hydrate! from stream to lake
Music city data Hail Hydrate! from stream to lake
 
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
Pulsar summit asia 2021   apache pulsar with mqtt for edge computingPulsar summit asia 2021   apache pulsar with mqtt for edge computing
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
 
Spark summit-east-dowling-feb2017-full
Spark summit-east-dowling-feb2017-fullSpark summit-east-dowling-feb2017-full
Spark summit-east-dowling-feb2017-full
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
 
DBCC 2021 - FLiP Stack for Cloud Data Lakes
DBCC 2021 - FLiP Stack for Cloud Data LakesDBCC 2021 - FLiP Stack for Cloud Data Lakes
DBCC 2021 - FLiP Stack for Cloud Data Lakes
 
Osacon 2021 hello hydrate! from stream to clickhouse with apache pulsar and...
Osacon 2021   hello hydrate! from stream to clickhouse with apache pulsar and...Osacon 2021   hello hydrate! from stream to clickhouse with apache pulsar and...
Osacon 2021 hello hydrate! from stream to clickhouse with apache pulsar and...
 
Codeless pipelines with pulsar and flink
Codeless pipelines with pulsar and flinkCodeless pipelines with pulsar and flink
Codeless pipelines with pulsar and flink
 
Interactive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using DruidInteractive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using Druid
 
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...
The Power of Intelligent Flows: Real-Time IoT Botnet Classification with Apac...
 
fluentd -- the missing log collector
fluentd -- the missing log collectorfluentd -- the missing log collector
fluentd -- the missing log collector
 
Matt Franklin - Apache Software (Geekfest)
Matt Franklin - Apache Software (Geekfest)Matt Franklin - Apache Software (Geekfest)
Matt Franklin - Apache Software (Geekfest)
 
Overview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics FrameworksOverview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics Frameworks
 
Machine Learning in the IoT with Apache NiFi
Machine Learning in the IoT with Apache NiFiMachine Learning in the IoT with Apache NiFi
Machine Learning in the IoT with Apache NiFi
 
ApacheCon 2021: Cracking the nut with Apache Pulsar (FLiP)
ApacheCon 2021:  Cracking the nut with Apache Pulsar (FLiP)ApacheCon 2021:  Cracking the nut with Apache Pulsar (FLiP)
ApacheCon 2021: Cracking the nut with Apache Pulsar (FLiP)
 
Cloud lunch and learn real-time streaming in azure
Cloud lunch and learn real-time streaming in azureCloud lunch and learn real-time streaming in azure
Cloud lunch and learn real-time streaming in azure
 
Scenic City Summit (2021): Real-Time Streaming in any and all clouds, hybrid...
Scenic City Summit (2021):  Real-Time Streaming in any and all clouds, hybrid...Scenic City Summit (2021):  Real-Time Streaming in any and all clouds, hybrid...
Scenic City Summit (2021): Real-Time Streaming in any and all clouds, hybrid...
 

Andere mochten auch

JustGiving – Serverless Data Pipelines, API, Messaging and Stream Processing
JustGiving – Serverless Data Pipelines,  API, Messaging and Stream ProcessingJustGiving – Serverless Data Pipelines,  API, Messaging and Stream Processing
JustGiving – Serverless Data Pipelines, API, Messaging and Stream ProcessingLuis Gonzalez
 
Multi-Datacenter Kafka - Strata San Jose 2017
Multi-Datacenter Kafka - Strata San Jose 2017Multi-Datacenter Kafka - Strata San Jose 2017
Multi-Datacenter Kafka - Strata San Jose 2017Gwen (Chen) Shapira
 
Spark and MapR Streams: A Motivating Example
Spark and MapR Streams: A Motivating ExampleSpark and MapR Streams: A Motivating Example
Spark and MapR Streams: A Motivating ExampleIan Downard
 
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...confluent
 
Data Pipelines Made Simple with Apache Kafka
Data Pipelines Made Simple with Apache KafkaData Pipelines Made Simple with Apache Kafka
Data Pipelines Made Simple with Apache Kafkaconfluent
 
A primer on building real time data-driven products
A primer on building real time data-driven productsA primer on building real time data-driven products
A primer on building real time data-driven productsLars Albertsson
 
Getting Started with Alluxio + Spark + S3
Getting Started with Alluxio + Spark + S3Getting Started with Alluxio + Spark + S3
Getting Started with Alluxio + Spark + S3Alluxio, Inc.
 
Tapjoy: Building a Real-Time Data Science Service for Mobile Advertising
Tapjoy: Building a Real-Time Data Science Service for Mobile AdvertisingTapjoy: Building a Real-Time Data Science Service for Mobile Advertising
Tapjoy: Building a Real-Time Data Science Service for Mobile AdvertisingSingleStore
 
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017Alluxio, Inc.
 
How to Become a Thought Leader in Your Niche
How to Become a Thought Leader in Your NicheHow to Become a Thought Leader in Your Niche
How to Become a Thought Leader in Your NicheLeslie Samuel
 
R, Scikit-Learn and Apache Spark ML - What difference does it make?
R, Scikit-Learn and Apache Spark ML - What difference does it make?R, Scikit-Learn and Apache Spark ML - What difference does it make?
R, Scikit-Learn and Apache Spark ML - What difference does it make?Villu Ruusmann
 
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017MLconf
 
Introduction to ML with Apache Spark MLlib
Introduction to ML with Apache Spark MLlibIntroduction to ML with Apache Spark MLlib
Introduction to ML with Apache Spark MLlibTaras Matyashovsky
 
3Com ETHERLINK II TP I
3Com ETHERLINK II TP I3Com ETHERLINK II TP I
3Com ETHERLINK II TP Isavomir
 
Capitulo iv El valor de la persona
Capitulo iv El valor de la personaCapitulo iv El valor de la persona
Capitulo iv El valor de la personaClaudia BC
 
Личностно-ориентированные технологии обучения
Личностно-ориентированные технологии обученияЛичностно-ориентированные технологии обучения
Личностно-ориентированные технологии обучения999Max999
 
Isro forbs web app
Isro forbs web appIsro forbs web app
Isro forbs web appappsme
 

Andere mochten auch (20)

JustGiving – Serverless Data Pipelines, API, Messaging and Stream Processing
JustGiving – Serverless Data Pipelines,  API, Messaging and Stream ProcessingJustGiving – Serverless Data Pipelines,  API, Messaging and Stream Processing
JustGiving – Serverless Data Pipelines, API, Messaging and Stream Processing
 
Multi-Datacenter Kafka - Strata San Jose 2017
Multi-Datacenter Kafka - Strata San Jose 2017Multi-Datacenter Kafka - Strata San Jose 2017
Multi-Datacenter Kafka - Strata San Jose 2017
 
Spark and MapR Streams: A Motivating Example
Spark and MapR Streams: A Motivating ExampleSpark and MapR Streams: A Motivating Example
Spark and MapR Streams: A Motivating Example
 
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...
Strata+Hadoop 2017 San Jose - The Rise of Real Time: Apache Kafka and the Str...
 
Data Pipelines Made Simple with Apache Kafka
Data Pipelines Made Simple with Apache KafkaData Pipelines Made Simple with Apache Kafka
Data Pipelines Made Simple with Apache Kafka
 
A primer on building real time data-driven products
A primer on building real time data-driven productsA primer on building real time data-driven products
A primer on building real time data-driven products
 
Getting Started with Alluxio + Spark + S3
Getting Started with Alluxio + Spark + S3Getting Started with Alluxio + Spark + S3
Getting Started with Alluxio + Spark + S3
 
Tapjoy: Building a Real-Time Data Science Service for Mobile Advertising
Tapjoy: Building a Real-Time Data Science Service for Mobile AdvertisingTapjoy: Building a Real-Time Data Science Service for Mobile Advertising
Tapjoy: Building a Real-Time Data Science Service for Mobile Advertising
 
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017
Effective Spark with Alluxio at Strata+Hadoop World San Jose 2017
 
How to Become a Thought Leader in Your Niche
How to Become a Thought Leader in Your NicheHow to Become a Thought Leader in Your Niche
How to Become a Thought Leader in Your Niche
 
R, Scikit-Learn and Apache Spark ML - What difference does it make?
R, Scikit-Learn and Apache Spark ML - What difference does it make?R, Scikit-Learn and Apache Spark ML - What difference does it make?
R, Scikit-Learn and Apache Spark ML - What difference does it make?
 
Yace 3.0
Yace 3.0Yace 3.0
Yace 3.0
 
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017
Yi Wang, Tech Lead of AI Platform, Baidu, at MLconf 2017
 
Introduction to ML with Apache Spark MLlib
Introduction to ML with Apache Spark MLlibIntroduction to ML with Apache Spark MLlib
Introduction to ML with Apache Spark MLlib
 
Spark Tips & Tricks
Spark Tips & TricksSpark Tips & Tricks
Spark Tips & Tricks
 
3Com ETHERLINK II TP I
3Com ETHERLINK II TP I3Com ETHERLINK II TP I
3Com ETHERLINK II TP I
 
Presentación
PresentaciónPresentación
Presentación
 
Capitulo iv El valor de la persona
Capitulo iv El valor de la personaCapitulo iv El valor de la persona
Capitulo iv El valor de la persona
 
Личностно-ориентированные технологии обучения
Личностно-ориентированные технологии обученияЛичностно-ориентированные технологии обучения
Личностно-ориентированные технологии обучения
 
Isro forbs web app
Isro forbs web appIsro forbs web app
Isro forbs web app
 

Ähnlich wie Design and Implementation of Spark Streaming Connectors for Event Hubs

Kappa Architecture on Apache Kafka and Querona: datamass.io
Kappa Architecture on Apache Kafka and Querona: datamass.ioKappa Architecture on Apache Kafka and Querona: datamass.io
Kappa Architecture on Apache Kafka and Querona: datamass.ioPiotr Czarnas
 
Sector Sphere 2009
Sector Sphere 2009Sector Sphere 2009
Sector Sphere 2009lilyco
 
sector-sphere
sector-spheresector-sphere
sector-spherexlight
 
Stream Analytics
Stream Analytics Stream Analytics
Stream Analytics Franco Ucci
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Databricks
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedMichael Spector
 
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LME
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LMESet your Data in Motion with Confluent & Apache Kafka Tech Talk Series LME
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LMEconfluent
 
Confluent kafka meetupseattle jan2017
Confluent kafka meetupseattle jan2017Confluent kafka meetupseattle jan2017
Confluent kafka meetupseattle jan2017Nitin Kumar
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Spark Summit
 
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
 
Introduction to Stream Processing
Introduction to Stream ProcessingIntroduction to Stream Processing
Introduction to Stream ProcessingGuido Schmutz
 
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Provectus
 
Harbour IT & VMware - vForum 2010 Wrap
Harbour IT & VMware - vForum 2010 WrapHarbour IT & VMware - vForum 2010 Wrap
Harbour IT & VMware - vForum 2010 WrapHarbourIT
 
Reactive programming
Reactive programmingReactive programming
Reactive programmingSUDIP GHOSH
 
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre Roman
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre RomanSpring Boot & Spring Cloud on Pivotal Application Service - Alexandre Roman
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre RomanVMware Tanzu
 
Spark and machine learning in microservices architecture
Spark and machine learning in microservices architectureSpark and machine learning in microservices architecture
Spark and machine learning in microservices architectureStepan Pushkarev
 
Spark Development Lifecycle at Workday - ApacheCon 2020
Spark Development Lifecycle at Workday - ApacheCon 2020Spark Development Lifecycle at Workday - ApacheCon 2020
Spark Development Lifecycle at Workday - ApacheCon 2020Pavel Hardak
 
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020Apache Spark Development Lifecycle @ Workday - ApacheCon 2020
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020Eren Avşaroğulları
 

Ähnlich wie Design and Implementation of Spark Streaming Connectors for Event Hubs (20)

Kappa Architecture on Apache Kafka and Querona: datamass.io
Kappa Architecture on Apache Kafka and Querona: datamass.ioKappa Architecture on Apache Kafka and Querona: datamass.io
Kappa Architecture on Apache Kafka and Querona: datamass.io
 
Sector Sphere 2009
Sector Sphere 2009Sector Sphere 2009
Sector Sphere 2009
 
sector-sphere
sector-spheresector-sphere
sector-sphere
 
Stream Analytics
Stream Analytics Stream Analytics
Stream Analytics
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...
 
Amazon Kinesis
Amazon KinesisAmazon Kinesis
Amazon Kinesis
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics Revised
 
Real time analytics
Real time analyticsReal time analytics
Real time analytics
 
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LME
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LMESet your Data in Motion with Confluent & Apache Kafka Tech Talk Series LME
Set your Data in Motion with Confluent & Apache Kafka Tech Talk Series LME
 
Confluent kafka meetupseattle jan2017
Confluent kafka meetupseattle jan2017Confluent kafka meetupseattle jan2017
Confluent kafka meetupseattle jan2017
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
 
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
 
Introduction to Stream Processing
Introduction to Stream ProcessingIntroduction to Stream Processing
Introduction to Stream Processing
 
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
 
Harbour IT & VMware - vForum 2010 Wrap
Harbour IT & VMware - vForum 2010 WrapHarbour IT & VMware - vForum 2010 Wrap
Harbour IT & VMware - vForum 2010 Wrap
 
Reactive programming
Reactive programmingReactive programming
Reactive programming
 
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre Roman
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre RomanSpring Boot & Spring Cloud on Pivotal Application Service - Alexandre Roman
Spring Boot & Spring Cloud on Pivotal Application Service - Alexandre Roman
 
Spark and machine learning in microservices architecture
Spark and machine learning in microservices architectureSpark and machine learning in microservices architecture
Spark and machine learning in microservices architecture
 
Spark Development Lifecycle at Workday - ApacheCon 2020
Spark Development Lifecycle at Workday - ApacheCon 2020Spark Development Lifecycle at Workday - ApacheCon 2020
Spark Development Lifecycle at Workday - ApacheCon 2020
 
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020Apache Spark Development Lifecycle @ Workday - ApacheCon 2020
Apache Spark Development Lifecycle @ Workday - ApacheCon 2020
 

Kürzlich hochgeladen

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
 
Generative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusGenerative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusTimothy Spann
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFxolyaivanovalion
 
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girl
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girlCall Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girl
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girlkumarajju5765
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
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
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023ymrp368
 
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
 
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
 
Capstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics ProgramCapstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics ProgramMoniSankarHazra
 
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% SecurePooja Nehwal
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxolyaivanovalion
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 
Zuja dropshipping via API with DroFx.pptx
Zuja dropshipping via API with DroFx.pptxZuja dropshipping via API with DroFx.pptx
Zuja dropshipping via API with DroFx.pptxolyaivanovalion
 
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...Pooja Nehwal
 
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
 
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
 

Kürzlich hochgeladen (20)

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
 
Generative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusGenerative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and Milvus
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girl
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girlCall Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girl
Call Girls 🫤 Dwarka ➡️ 9711199171 ➡️ Delhi 🫦 Two shot with one girl
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
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
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023
 
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
 
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
 
Capstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics ProgramCapstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics Program
 
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
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptx
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 
Zuja dropshipping via API with DroFx.pptx
Zuja dropshipping via API with DroFx.pptxZuja dropshipping via API with DroFx.pptx
Zuja dropshipping via API with DroFx.pptx
 
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
 
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
 
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...
 

Design and Implementation of Spark Streaming Connectors for Event Hubs

  • 1. Design and Implementation of Spark Streaming Connectors ARIJIT TARAFDAR NAN ZHU MARCH 23,2017
  • 2. About Us Arijit Tarafdar Software Engineer@Azure HDInsight. Work on Spark Streaming/Structured Streaming service in Azure. Committee Member of XGBoost@DMLC and Apache MxNet (incubator). Spark Contributor. Known as CodingCat in GitHub. Nan Zhu Software Engineer@Azure HDInsight. Work on Spark/Spark Streaming on Azure. Previously worked with other distributed platforms like DryadLinq and MPI. Also worked on graph coloring algorithms which was contributed to ADOL-C (https://projects.coin-or.org/ADOL-C).
  • 3. Real Time Data Analytics Results Processing Engine Continuous Data Source Control Manager Continuous Data Source API Persistent Data Storage Layer Spark Streaming, Structured Streaming Deliver real time data to Spark at scale Real time view of data (message queue or files filtered by timestamp) Blobs/Queues/Tables/Files Continuous Application Architecture and Role of Spark Connectors Not only size of data is increasing, but also the velocity of data ◦ Sensors, IoT devices, social networks and online transactions are all generating data that needs to be monitored constantly and acted upon quickly.
  • 4. Outline •Recap of Spark Streaming •Introduction to Event Hubs •Connecting Azure Event Hubs and Spark Streaming •Design Considerations for Spark Streaming Connector •Contributions Back to Community •Future Work
  • 5. Spark Streaming - Background Task 1 Task 2 Task L RDD 1 @ t RDD 1 @ t-1 RDD 1 @ 0 Stream 1 Task 1 Task 2 Task M RDD N @ t RDD N @ t-1 RDD N @ 0 Stream N Micro Batch @ t Task 1 Task 2 Task L Task 1 Task 2 Task M Window Duration Batch Duration
  • 6. Azure Event Hubs - Introduction Partition 1 Partition 2 Partition J Event Hubs 1 Partition 1 Partition 2 Partition K Event Hubs L Event Hubs Namespace 1 Partition 1 Partition 2 Partition K Event Hubs 1 Partition 1 Partition 2 Partition P Event Hubs N Event Hubs Namespace M
  • 7. Azure Event Hubs - Introduction https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-what-is-event-hubs
  • 8. Data Flow – Event Hubs • Proactive message delivery • Efficient in terms of communication cost • Data source treated as commit log of events • Events read in batch per receive() call New Old Event Hubs Partition (Event Hubs Server) Prefetch Queue (Event Hubs Client) Streaming Application
  • 9. Event Hubs – Offset Management • Event Hubs expect offset management to be performed on the receiver side • Spark streaming uses DFS based persistent store (HDFS, ADLS, etc.) • Offset is stored per consumer group per partition per event hubs per event hubs namespace /* An interface to read/write offset for a given Event Hubs namespace/name/partition */ @SerialVersionUID(1L) trait OffsetStore extends Serializable { def open(): Unit def write(offset: String): Unit def read() : String def close(): Unit }
  • 10. First Version: Receiver-based Spark Streaming Connector for Azure Event Hub
  • 11. Application Driver (Spark) Receiver Executor (Spark) Streaming Context Spark Context Eventhubs Receiver Task Executor (Spark) User Defined Functions ADLS ADLS Write Ahead Log (WAL) Checkpoint Directory Memory Block Data Block Metadata Jobs Tasks Checkpoint Data Azure Eventhubs Input Stream ADLS WASB Output Storage Fault Tolerance – Spark Receiver Based Event Hubs Connector
  • 12. Restarted Application Driver (Spark) Restarted Receiver Executor (Spark) Restarted Streaming Context Restarted Spark Context Restarted Eventhubs Receiver Restarted Task Executor (Spark) User Defined Functions ADLS ADLS Write Ahead Log (WAL) Checkpoint Directory Memory Recover Block Data Recover Block Metadata Jobs Tasks Restart Computation From Checkpoint Data Azure Eventhubs ADLS WASB Output Storage Spark Streaming – Recovery After Failure
  • 13. Event Hubs Receiver – Class Signature private[eventhubs] class EventHubsReceiver( eventhubsParams: Map[String, String], partitionId: String, storageLevel: StorageLevel, offsetStore: Option[OffsetStore], receiverClient: EventHubsClientWrapper, maximumEventRate: Int) extends Receiver[Array[Byte]](storageLevel) with Logging { ... }
  • 14. Event Hubs Receiver – Methods Used/Implemented @DeveloperApi abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { def onStart(): Unit def onStop(): Unit def store(dataItem: T) { supervisor.pushSingle(dataItem) } def store(dataBuffer: ArrayBuffer[T]) { supervisor.pushArrayBuffer(dataBuffer, None, None) } def restart(message: String, error: Throwable) { supervisor.restartReceiver(message, Some(error)) } }
  • 15. Azure Event Hubs/Spark Expectations Receiver-based Connection (Event Hubs) Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks (Spark) Logging Data Before Ack WAL/Spark Checkpoint (Event Hubs) Client-side Offset Management Offset Store A Natural Fit!!! Why Receiver based connector?
  • 16. Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Lessons learnt from based connector?
  • 17. Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Logging Data Before Ack WAL/Spark Checkpoint Performance/Data loss due to Spark bug/No easy recovery from code update https://issues.apache.org/jira/browse/SPARK-18957 Lessons learnt from based connector?
  • 18. Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Logging Data Before Ack WAL/Spark Checkpoint Performance/Data loss due to Spark bug/No easy recovery from code update Client-side Offset Management Offset Store Looks fine…. Lessons learnt from based connector?
  • 19. Bridging Spark Streaming and Event Hubs WITHOUT Receiver How the Idea Extends to Other Data Sources (in Azure & Your IT Infrastructure)?
  • 20. Extra Resources Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Fault tolerance Mechanism WAL/Spark Checkpoint Perf./Data Loss due to Spark Bug/No Recovery from Code Update Client-side Offset Management Offset Store Looks fine….
  • 21. From Event Hubs to General Data Sources (1) •Communication Pattern • Azure Event Hubs: Long-Running Receiver, Proactive Data Delivery • Kafka: Receiver Start/Shutdown in a free-style, Passive Data Delivery Most Critical Factor in Designing a Resource-Efficient Spark Streaming Connector!
  • 22. Tackling Extra Resource Requirement Azure EventHubs EvH- Namespace-1 EventHub-1 P1 PN . . . Reduce Resource Requirements: Customized Receiver Logic User-Defined Lambdas EventHubsRDD .map() MapPartitionsRDD Spark Tasks Compact Data Receiving and Processing in the same Task Inspired by Kafka Direct DStream! Being More Challenging with a Different Communication Pattern!
  • 23. Bridging Spark Execution Model and Communication Pattern Expectation Azure EventHubs EvH- Namespace-1 EventHub-1 P1 PN . . . Customized Receiver Logic User-Defined Lambdas EventHubsRDD .map() MapPartitionsRDD Spark Task Passive Message Delivery Layer Recv(expectedMsgNum: Int) – Blocking API Long-running/Proactive Receiver expected by Event Hubs vs. Transient Tasks started for each Batch by Spark
  • 24. Takeaways (1) Requirements in Event Hubs Receiver-based Connection Problems Solution Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Compact Data Receiving/Processi ng, with the facilitates from Passive Message Delivery Communication Pattern in Data Sources Plays the Key Role in Resource-Efficient Design of Spark Streaming Connector
  • 26. Fault Tolerance Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Fault tolerance Mechanism WAL/Spark Checkpoint Perf./Data Loss due to Spark Bug/No Recovery from Code Update Client-side Offset Management Offset Store Looks fine….
  • 27. From Event Hubs to General Data Sources (2) •Fault-Tolerance • Capability • Guarantee graceful recovery (no data loss, recover from where you stopped, etc.) with application stops for various reasons • Efficiency • Minimum impact to application performance and user deployment
  • 28. …RDD L-t RDD L-(t-1) RDD L-0 Stream L Unexpected Application Stop Checkpoint Time RDD L-(t-1)RDD L-t Recovery From Checkpoint, or Re-evaluated Capability – Recover from unexpected stop
  • 29. …RDD L-(t-1) RDD L-0 Stream L Application Upgrade … Application Stop Spark Checkpoint Mechanism Serializes Everything and does not recognize a re-compiled class Capability – Recover from planned stop RDD L-(2t) Resume Application with updated Implementation Fetch the latest offset Offset Store Your Connector shall maintain this!!!
  • 30. Efficiency - What to be Contained in Checkpoint Files? • Checkpointing takes your computing resources!!! • Received Event Data • too large • The range of messages to be processed in each batch • Small enough to quickly persist data Azure EventHubs EvH- Namespace-1 EventHub-1 P1 PN . . . EventHubsRDD .map() MapPartitionsRDD Passive Message Delivery Layer Recv(expectedMsgNum: Int) – Blocking API Persist this mapping relationship, i.e. using EventHubs itself as data backup
  • 31. Efficiency - Checkpoint Cleanup •Connectors for Data Source Requiring Client- side offset management generates Data/Files for each Batch • You have to clean up SAFELY • Keep recovery feasible • Coordinate with Spark’s checkpoint process • Override clearCheckpointData() in EventHubsDStream (our implementation of Dstream) • Triggered by Batch Completion • Delete all offset records out of the remembering window
  • 32. Takeaways (2) Requirements in Event Hubs Receiver-based Connection Problems Solution Fault tolerance Mechanism WAL/Spark Checkpoint Perf./Data Loss due to Spark Bug/No Recovery from Code Update Checkpoint Mapping Relationship instead of Data/Self- management Offset Store/Coordinate Checkpoint Cleanup Fault Tolerance Design is about Interaction with Spark Streaming Checkpoint
  • 33. No Problem any more?
  • 34. Offset Management Requirements in Event Hubs Receiver-based Connection Problems Long-Running Receiver/Proactive Message Fetching Long-Running Receiver Tasks Extra Resource Requirements Fault tolerance Mechanism WAL/Spark Checkpoint Data Loss due to Spark Bug Client-side Offset Management Offset Store Looks fine…. Is it really fine???
  • 35. From Event Hubs to General Data Sources (3) •Message Addressing Rate Control
  • 36. Message Addressing • Why Message Addressing? • When creating a client instance of data source in a Spark task, where to start receiving? • Without this info, you have to replay the stream for every newly created client Data Source Client Start from the first msg Data Source Client Start from where? • Design Options: • Xth message (X: 0, 1, 2, 3, 4….) • server side metadata to map the message ID to the offset in storage system • Actual offset • Simpler server side design Fault Or Next Batch
  • 37. Rate Control • Why Rate Control • Prevent the messages flooding into the processing pipelines • e.g. just start processing a queued up data sources • Design Options • Number of messages: I want to consume 1000 messages in next batch • Assuming the homogeneous processing overhead • Size of messages: I want to receive at most 1000 bytes in next batch • Complicated Server side logic -> track the delivered size • Larger messages, longer processing time is not always true Data Source Client Start from the first msg Data Source Client Consume all messages at once? May crash your processing engine!!!A Long Stop!!!
  • 38. Kafka Choice • Message Addressing: • Xth message: 0, 1, 2, 3, 4, .. • Rate Control • Number of Messages: 0, 1, 2, 3, 4, … Driver Executor Executor Kafka Message Addressing and Rate Control: Batch 0: How many messages are to be processed in next batch, and where to start? 0 - 999 Batch 1: How many messages are to be processed in next batch, and where to start? 1000 - 1999
  • 39. Azure Event Hubs’ Choice • Message Addressing: • Offset of messages: 0, size of msg 0, size of (msg 0 + msg 1),… • Rate Control • Number of Messages: 0, 1, 2, 3, 4, … This brings totally different connector design/implementation!!!
  • 40. Distributed Information for Rate Control and Message Addressing Driver Executor Executor Rate Control: Batch 0: How many messages are to be processed in next batch, and where to start? 0 - 999 Azure EventHubs Batch 1: How many messages are to be processed in next batch, and where to start? 1000 - 1999 What’s the offset of 1000th message??? The answer appeared in Executor side (when Task receives the message (as part of message metadata)) Build a Channel to Pass Information from Executor to Driver!!!
  • 41. HDFS-based Channel Implementation LastOffset_P1_Batch_i LastOffset_PN_Batch_i EventHubsRDD Tasks .map() MapPartitionsRDD Tasks What’s the next step??? Simply let Driver-side logic read the files? •APIs like RDD.take(x) evaluates only some of the partitions…Batch 0 generate 3 files, and Batch 1 generates 5 files… •You have to merge the latest files with the historical results and commit and then direct the driver-side logic to read No!!!
  • 42. HDFS-based Channel Implementation LastOffset_P1_Batch_i LastOffset_PN_Batch_i EventHubsRDD Tasks .map() MapPartitionsRDD Tasks •APIs like RDD.take(x) evaluates only some of the partitions…Batch 0 generate 3 files, and Batch 1 generates 5 files… •You have to merge the latest files with the historical results and commit ... •Ensure that all streams’ offset are committed transactionally •Discard the partially merged/committed results to rerun the batch
  • 43. HDFS-based Channel Implementation RDD Generation “Thread” Job Execution “Thread” Generate RDD Blocking (wait) Processing RDD BatchComplete Event SparkListenerBus CustomizedListen er CommitOffsets and Notify
  • 44. HDFS-based Channel Implementation RDD Generation “Thread” Job Execution “Thread” Generate RDD Blocking (wait) Processing Micro Batch BatchComplete Event SparkListenerBus CustomizedListen er CommitOffsets and Notify DStream.graph: DStreamGraph
  • 45. Takeaways (3) • There are multiple options on the Server-side design for Message Addressing and Rate Control • To design and implement a Spark Streaming connector, you have to understand what are the options adopted in server side The key is the combination!!!
  • 46. Contribute Back to Community Failed Recovery from checkpoint caused by the multi-threads issue in Spark Streaming scheduler https://issues.apache.org/jira/browse/SPARK-19280 One Realistic Example of its Impact: You are potentially getting wrong data when you use Kafka and reduceByWindow and recover from a failure Data loss caused by improper post-batch-completed processing https://issues.apache.org/jira/browse/SPARK-18905 Inconsistent Behavior of Spark Streaming Checkpoint https://issues.apache.org/jira/browse/SPARK-19233
  • 47. Summary • Spark Streaming Connector for Azure Event Hubs enables the user to perform various types of analytics over streaming data from a fully managed, cloud-scale message telemetry ingestion service • https://github.com/hdinsight/spark-eventhubs • Design and Implementation of Spark Streaming Connectors • Coordinate Execution Model and Communication Pattern • Fault Tolerance (Spark Streaming Checkpoint v.s. self-managed fault tolerance facilitates) • Message Addressing and Rate Control (Server&Connector Co-Design) • Contributing Back to the Community • Microsoft is the organization with the most open source contributors in 2016!!! • http://www.businessinsider.com/microsoft-github-open-source-2016-9
  • 48. If you do not want to handle this complexity Move to Azure HDInsight…
  • 49. Future Work Structured Streaming integration with Event Hubs (will release at the end of month) Streaming Data Visualization with PowerBI (alpha released mode) Streaming ETL Solutions on Azure HDInsight!
  • 50. Thank You!!! Build a Powerful&Robust Data Analytic Pipeline with Spark@Azure HDInsight!!!

Hinweis der Redaktion

  1. Two types of datasets Bounded: Finite, unchanging datasets Unbounded: Infinite datasets that are appended to continuously Unbounded – data is generated all the time and we want to know now Glue between unbounded data source like event hubs and powerful processing engine like Spark Goal is to deliver near real time analysis or view.
  2. Micro-batching mechanism, processes continuous and infinite data source Batch scheduled at regular time interval or after certain number of events received Distributed Stream is the highest level abstraction over continuous creation and expiration of RDDs Batch duration – single RDD generated Window duration – multiple of batch duration, may use multiple RDDs RDDs contains partitions, one task per partitions
  3. High throughput, low latency offered as platform as a service on Azure No cluster set up required, no monitoring required User can concentrate only on ingress and egress of data Event hubs namespace collection of event hubs, an event hub is a collection of partitions, a partition is a sequential collection of events Up to 32 partitions per event hub but can be increased if required
  4. -HTTP or AMQP with transport level security (TLS/SSL) -HTTP has higher message transmission overhead -AMQP has higher connection setup overhead -Consumer group gives logical view of event hubs partitions, including addressing same partition at different offsets -Up to 20 consumer groups per event hubs -1 receiver per consumer group
  5. Each partition can be viewed as a commit log Event Hubs client maintains prefetch queue to proactively get messages from the server Receive call by application gets messages in batch from the prefetch queue to the caller.
  6. No support from Event Hubs server yet Offset is managed by the Event Hubs connector at the Spark application side Uses distributed file system like HDFS, ADLS, etc. Offset is stored per consumer group, per partition, per event hub, per event hub namespace Event hubs clients are initialized with an initial offset from the which event hubs will start sending data Offset is determined in one of three ways – start of stream, previously saved offset, enqueue time
  7. - How do we bridge
  8. Reliable receivers – received data backed up in a reliable persistent store (WAL), no data lost between application restarts Reliable receivers – offset saved after saving to persistent store and pushing to block manager Both executors and driver use the WAL
  9. On application restart data is processed from the WAL first up to the offset saved before the previous application stop Receiver tasks then start the event hubs clients, one per partition with the last offset saved for each partition.
  10. Describe each parameter. Extends spark provided Receiver class with specific type of Array of Bytes which is the exact content of the user data per event. Storage level whether to spill to disk when memory usage reaches capacity.
  11. On start establishes connections to event hubs On stop cleans up the connections Reliably store data to block manager Restart call stop and start.