Download - Seattle spark-meetup-032317

Transcript
Page 1: Seattle spark-meetup-032317

Design and Implementation of Spark Streaming Connectors

ARIJ IT TARAFDARNAN ZHUMARCH 23,2017

Page 2: Seattle spark-meetup-032317

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).

Page 3: Seattle spark-meetup-032317

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.

Page 4: Seattle spark-meetup-032317

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

Page 5: Seattle spark-meetup-032317

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

Page 6: Seattle spark-meetup-032317

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

Page 7: Seattle spark-meetup-032317

Azure Event Hubs - Introduction

https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-what-is-event-hubs

Page 8: Seattle spark-meetup-032317

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

Page 9: Seattle spark-meetup-032317

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}

Page 10: Seattle spark-meetup-032317

First Version: Receiver-based Spark Streaming Connector for

Azure Event Hub

Page 11: Seattle spark-meetup-032317

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

ADLSWASB

Output Storage

Fault Tolerance – Spark Receiver Based Event Hubs Connector

Page 12: Seattle spark-meetup-032317

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

ADLSWASB

Output Storage

Spark Streaming – Recovery After Failure

Page 13: Seattle spark-meetup-032317

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 { ... }

Page 14: Seattle spark-meetup-032317

Event Hubs Receiver – Methods Used/Implemented

@DeveloperApiabstract 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)) }}

Page 15: Seattle spark-meetup-032317

Azure Event Hubs/Spark Expectations Receiver-based Connection

(Event Hubs) Long-Running Receiver/Proactive Message

FetchingLong-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?

Page 16: Seattle spark-meetup-032317

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?

Page 17: Seattle spark-meetup-032317

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?

Page 18: Seattle spark-meetup-032317

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

updateClient-side Offset

Management Offset Store Looks fine….

Lessons learnt from based connector?

Page 19: Seattle spark-meetup-032317

Bridging Spark Streaming and Event Hubs WITHOUT Receiver

How the Idea Extends to Other Data Sources (in Azure & Your IT

Infrastructure)?

Page 20: Seattle spark-meetup-032317

Extra ResourcesRequirements in

Event HubsReceiver-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….

Page 21: Seattle spark-meetup-032317

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!

Page 22: Seattle spark-meetup-032317

Tackling Extra Resource Requirement

Azure EventHubsEvH-

Namespace-1

EventHub-1

P1

PN

.

.

.

Reduce Resource Requirements:

Customized ReceiverLogic

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!

Page 23: Seattle spark-meetup-032317

Bridging Spark Execution Model and Communication Pattern Expectation

Azure EventHubsEvH-

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

Page 24: Seattle spark-meetup-032317

Takeaways (1)

Requirements in Event Hubs

Receiver-based Connection Problems Solution

Long-Running Receiver/Proactiv

e 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

Page 25: Seattle spark-meetup-032317

Next Problem…

Page 26: Seattle spark-meetup-032317

Fault ToleranceRequirements in

Event HubsReceiver-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….

Page 27: Seattle spark-meetup-032317

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

Page 28: Seattle spark-meetup-032317

…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

Page 29: Seattle spark-meetup-032317

…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!!!

Page 30: Seattle spark-meetup-032317

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 EventHubsEvH-

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

Page 31: Seattle spark-meetup-032317

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

Page 32: Seattle spark-meetup-032317

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

Page 33: Seattle spark-meetup-032317

No Problem any more?

Page 34: Seattle spark-meetup-032317

Offset ManagementRequirements in

Event HubsReceiver-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???

Page 35: Seattle spark-meetup-032317

From Event Hubs to General Data Sources (3)•Message Addressing Rate Control

Page 36: Seattle spark-meetup-032317

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 SourceClient

Start from the first msg

Data SourceClient

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

FaultOr

Next Batch

Page 37: Seattle spark-meetup-032317

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 SourceClient

Start from the first msg

Data SourceClient

Consume all messages at once? May crash your processing engine!!!A Long Stop!!!

Page 38: Seattle spark-meetup-032317

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

Page 39: Seattle spark-meetup-032317

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!!!

Page 40: Seattle spark-meetup-032317

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!!!

Page 41: Seattle spark-meetup-032317

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!!!

Page 42: Seattle spark-meetup-032317

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

Page 43: Seattle spark-meetup-032317

HDFS-based Channel Implementation

RDD Generation “Thread” Job Execution “Thread”

Generate RDD

Blocking (wait)

Processing RDD

BatchComplete Event

SparkListenerBus

CustomizedListener

CommitOffsets and Notify

Page 44: Seattle spark-meetup-032317

HDFS-based Channel Implementation

RDD Generation “Thread” Job Execution “Thread”

Generate RDD

Blocking (wait)

Processing Micro Batch

BatchComplete Event

SparkListenerBus

CustomizedListener

CommitOffsets and Notify

DStream.graph: DStreamGraph

Page 45: Seattle spark-meetup-032317

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!!!

Page 46: Seattle spark-meetup-032317

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

Page 47: Seattle spark-meetup-032317

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

Page 48: Seattle spark-meetup-032317

If you do not want to handle this complexityMove to Azure HDInsight…

Page 49: Seattle spark-meetup-032317

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!

Page 50: Seattle spark-meetup-032317

Thank You!!!Build a Powerful&Robust Data Analytic Pipeline with Spark@Azure HDInsight!!!