SlideShare ist ein Scribd-Unternehmen logo
1 von 38
Downloaden Sie, um offline zu lesen
Streaming Data Lakes Using
Kafka Connect +Apache Hudi
Balaji Varadarajan, Vinoth Chandar
Speakers
Vinoth Chandar
PMC Chair/Creator of Hudi
Sr.Staff Eng @ Uber (Data
Infra/Platforms, Networking)
Principal Eng @ Confluent (ksqlDB,
Kafka/Streams)
Staff Eng @ Linkedin (Voldemort,
DDS)
Sr Eng @ Oracle
(CDC/Goldengate/XStream)
Balaji Varadarajan
PMC Member, Apache Hudi
Sr. Staff Eng @ Robinhood, Data Infra
Tech Lead @Uber, Data Platform
Staff Engineer @Linkedin, Databus CDC
Agenda
1) Background
2) Hudi 101
3) Hudi’s Spark Writers (existing)
4) Kafka Connect Sink (new)
5) Onwards
Background
Event Streams, Data Lakes
Data Lakes are now essential
Architectural Pattern for Analytical Data
❏ Data Lake != Spark, Flink
❏ Data Lake != Files on S3
❏ Raw data (OLTP schema)
❏ Derived Data (OLAP/BI, ML schema)
Open Storage + Scalable Compute
❏ Avoid data lock-in, Open formats (data +
metadata)
❏ Efficient, Universal (Analytics, Data
Science)
Lot of exciting progress
❏ Lakehouse = Lake + Warehouse
❏ Data meshes on Lakes => Need for streams
Source:
https://martinfowler.com/bliki/images/dataLake/context.png
Event Streams are the new norm
Events come in many flavors
Database change Events
❏ High fidelity, High value, update/deletes
❏ E.g: Debezium changelogs into Kafka
Application/Service business events
❏ High volume, Immutable or Deltas,
❏ E.g: Emit Uber app events, emit changes from IoT sensors
SaaS Data Sources
❏ Lower volume, mutable
❏ E.g: polling Github events API
Database
Kafka
Cluster
Apps/
Services
Event Firehose
External
Sources
Extracting Event Streams
Kafka
Connect
Sources
Why not just Connect File Sinks?
Queries
DFS/Cloud Storage
Data Lake??
Files
Kafka
Cluster
Kafka
Connect
Sinks
(S3/HDFS)
Challenges
Working at the file abstraction level is painful
❏ Transactional, Concurrency Control
❏ Updates subset of data, indexing for faster access
Scalability, Operational Overhead
❏ Writing columnar files is resource intensive
❏ Partitioned data increases memory overhead
Lack of management
❏ Control file sizes, Deletes for GDPR/Compliance
❏ Re-align storage for better query performance
Apache Hudi
Transactional Writes, MVCC/OCC
❏ Work with tables and records
❏ Automatic compaction, clustering, sizing
First class support for Updates, Deletes
❏ Record level Update/Deletes inspired by stream
processors
CDC Streams From Lake Storage
❏ Storage Layout optimized for incremental fetches
❏ Hudi’s unique contribution in the space
Hudi 101
Components, APIs, Architecture
Stream processing + Batch data
The Incremental Stack
+ Intelligent, Incremental
+ Fast, Efficient
+ Scans, Columnar formats
+ Scalable Compute
https://www.oreilly.com/content/ubers-case-for-
incremental-processing-on-hadoop/; 2016
The Hudi Stack
❏ Complete “data” lake platform
❏ Tightly integrated, Self managing
❏ Write using Spark, Flink
❏ Query using Spark, Flink, Hive,
Presto, Trino, Impala, AWS
Athena/Redshift, Aliyun DLA etc
❏ Out-of-box tools/services for data ops
http://hudi.apache.org/blog/2021/07/21/st
reaming-data-lake-platform
Storage Layout
❏ Powers arguably the largest transactional data
lake on the planet @ Uber
❏ (Database CDC) Robinhood’s near-realtime
data lake
❏ (ML Feature stores) @ Logical Clocks
❏ (Event Deletions/De-Duping) @ Moveworks
❏ Many more companies, pre-installed by 5
major cloud providers
1000+
Slack members
150+
Contributors
1000+
GH Engagers
~10-20
PRs/week
20+
Committers
10+
PMCs
The Community
Hudi DeltaStreamer
Efficient, Micro-batched
Event
Streams
DFS/Cloud Storage
Tables
Pull using Spark
Kafka
De-Dupe Indexing
Txn
DeltaStreamer Utility,
Spark Streaming
Cluster
Optimize
Compact
Apply
Pull
Cleaning
Current Kafka to Hudi Options
- Ingest streaming data to Data Lake - Raw Tables
- Current Solutions through Spark:
- Hudi DeltaStreamer
- Spark Structured Streaming
Kafka
Cluster
Hudi
DeltaStreamer
Spark Structured
Streaming
DFS/Cloud
Storage
Tables
Apply
Structured Streaming Sink
// Read data from stream
Dataset<Row> streamingInput = spark.readStream()...
// Write to Hudi in a streaming fashion
DataStreamWriter<Row> writer = streamingInput.writeStream()
.format("org.apache.hudi")
.option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType)
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "timestamp")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
.option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());
String tablePath = “s3://…."
// Schedule the job
StreamingQuery query = ...
writer.trigger(Trigger.ProcessingTime(500)).start(tablePath);
query.awaitTermination(streamingDurationInMs);
DeltaStreamer Utility
❏ Fully Managed Ingestion
and ETL service
❏ Integration with various
Streaming and batch
sources
❏ Table State &
Checkpoints
transactionally consistent
❏ Pluggable
Transformations for ETL
use cases.
DeltaStreamer Example
spark-submit
--master yarn
--packages org.apache.hudi:hudi-utilities-bundle_2.12:0.8.0
--class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer
--conf spark.scheduler.mode=FAIR
--conf spark.task.maxFailures=5
...
--enable-sync
--hoodie-conf auto.offset.reset=latest
--hoodie-conf hoodie.avro.schema.validate=true
….
--table-type MERGE_ON_READ
--source-class org.apache.hudi.utilities.sources.AvroKafkaSource
--schemaprovider-class org.apache.hudi.utilities.schema.SchemaRegistryProvider
--props /path/job.properties
--transformer-class com.some.someTransformer
--continuous ← Enables async compaction, clustering & cleaning along with streaming writes
Streaming Data Lake without writing any code!
Case Study: Robinhood Data Lake
Master
RDS
Replica
RDS
Table Topic
DeltaStreamer
(Live)
DeltaStreamer
(Bootstrap)
DATA LAKE
(s3://xxx/…
Update schema
and partition
Write incremental data
and checkpoint offsets
Case Study: Robinhood Data Lake
❏ 1000s of CDC based Streaming ingest pipelines supported by Apache Hudi
DeltaStreamer.
❏ Data Lake freshness Latency down to 5-15 mins from hours.
❏ Powers critical dashboards and use-cases
End-to-End Streaming Data Lake
❏ Data Lake has both raw tables and derived tables built through ETLs.
❏ Streaming Data-lake - Needs streaming semantics supported for both kinds of tables.
❏ The Missing Primitive : Derived Tables need Changelog view of the upstream dataset ->
Apache Hudi Incremental Read to rescue
The Big Picture
Pull
Database Event
Streams
Apps/
Service
s
External
Sources
CDC
Push
Streaming Data Lake
Raw Tables
DeltaStreamer
Spark Streaming Hudi Change log
Derived Tables
DeltaStreamer
Spark
Streaming
Connect Hudi Sink
Kafkaesque, Commit protocol, Transactional
Motivations
Integration with Kafka Connect
❏ Separation of concerns (writing vs optimization/management)
❏ Streamline operationally, just one framework for ingesting
❏ Less need for Spark expertise
Faster data
❏ Amortize startup costs (containers, queue delays)
❏ Commit frequently i.e every 1 minute (every N secs in near
future)
❏ E.g avro records in Kafka log to Hudi’s log format
Putting it all together
Event
Streams
DFS/Cloud Storage
Tables
Kafka
De-Dupe
Indexing Txn
Hudi Connect Sink
(Writing)
Commit
Pull
Compact
Cluster
Hudi’s Table Services
(Optimization, management)
Clean
Deletes
Design Challenges
Determining Transaction Boundaries
❏ No co-ordination via driver process like Spark/Flink
❏ Workers doing their own commits => horrible
concurrency bottlenecks
Connect APIs cannot express DAGs
❏ Meant to be simple `putRecords()`/`preCommit()`
❏ Indexing, De-duplication, Storage optimization all
shuffle data
Design Overview
Central Transaction Co-ordination
❏ Use Kafka to elect co-
ordinator.
❏ Runs in one of the workers
Kafka as control channel
❏ Consume from latest control
topic offsets
https://cwiki.apache.org/confluence/display/HUDI/RFC-32+Kafka+Connect+Sink+for+Hudi
Design Overview
Transaction Coordinator
❏ Daemon thread on owner of
partition 0
❏ Sends commands to participants
Embedded Hudi Java Writer
❏ Lands data into set of file groups,
mapped to a partition
❏ Hudi’s commit fencing guards
from failures/partial writes
Co-ordinator State Machine
Paxos-like two phase commit
❏ Co-ordinator process to start, end commits
❏ Safety > liveness, abort after timeout
Participants “pause” at each commit boundary
❏ Return latest write offsets to co-ordinator
❏ Resume again on start of next commit
Example Sink Configuration
# hudi table properties
target.base.path
target.table.name
target.database.name
schemaprovider.class
partition.field.name
hoodie.table.base.file.format
Pre-release, subject to change.
Refer to official Hudi docs, for actual config names.
# controller properties
control.topic.name
coordinator.writestatus.timeout
write.retry.timeout
Choosing Right
Delta Streamer Connect Sink
Provides full set of Hudi features Insert only for now, indexes/updates coming
as enhancements
Offers better elasticity for merging/writing
columnar data
i.e copy-on-write tables
Great impedance match with Kafka, for
landing avro/row-oriented data i.e merge-on-
read tables
Data freshness of several minutes, if not
running in continuous mode
Approach ~1 min freshness
Need experience with Spark/Flink Operate all data ingestion in a single
framework.
What’s to come
Onwards
Kafka + Hudi
Support for mutable, keyed updates/deletes
❏ Need to implement a new index ala Flink writer
❏ preCombine, buffering/batching
What if : Back Kafka’s tiered storage using Hudi
❏ Map offsets to Hudi commit_seq_no
❏ Columnar reads for historical/catch-up reads
Engage With Our Community
User Docs : https://hudi.apache.org
Technical Wiki : https://cwiki.apache.org/confluence/display/HUDI
Github : https://github.com/apache/hudi/
Twitter : https://twitter.com/apachehudi
Mailing list(s) : dev-subscribe@hudi.apache.org (send an empty email to subscribe)
dev@hudi.apache.org (actual mailing list)
Slack : https://join.slack.com/t/apache-hudi/signup
Questions?
Thanks!

Weitere ähnliche Inhalte

Was ist angesagt?

Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
HostedbyConfluent
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
Cloudera, Inc.
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
DataWorks Summit
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
DataWorks Summit
 

Was ist angesagt? (20)

Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Building robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and DebeziumBuilding robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and Debezium
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
Iceberg: A modern table format for big data (Strata NY 2018)
Iceberg: A modern table format for big data (Strata NY 2018)Iceberg: A modern table format for big data (Strata NY 2018)
Iceberg: A modern table format for big data (Strata NY 2018)
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the Covers
 
Presto
PrestoPresto
Presto
 
Making Data Timelier and More Reliable with Lakehouse Technology
Making Data Timelier and More Reliable with Lakehouse TechnologyMaking Data Timelier and More Reliable with Lakehouse Technology
Making Data Timelier and More Reliable with Lakehouse Technology
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 
Introduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse ArchitectureIntroduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse Architecture
 
Reshape Data Lake (as of 2020.07)
Reshape Data Lake (as of 2020.07)Reshape Data Lake (as of 2020.07)
Reshape Data Lake (as of 2020.07)
 
Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
Designing Apache Hudi for Incremental Processing With Vinoth Chandar and Etha...
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
 
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangApache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 

Ähnlich wie Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apache Software Foundation

Ch 22: Web Hosting and Internet Servers
Ch 22: Web Hosting and Internet ServersCh 22: Web Hosting and Internet Servers
Ch 22: Web Hosting and Internet Servers
webhostingguy
 
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
DataWorks Summit
 

Ähnlich wie Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apache Software Foundation (20)

Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
 
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...
 
[Pulsar summit na 21] Change Data Capture To Data Lakes Using Apache Pulsar/Hudi
[Pulsar summit na 21] Change Data Capture To Data Lakes Using Apache Pulsar/Hudi[Pulsar summit na 21] Change Data Capture To Data Lakes Using Apache Pulsar/Hudi
[Pulsar summit na 21] Change Data Capture To Data Lakes Using Apache Pulsar/Hudi
 
Introduction and HDInsight best practices
Introduction and HDInsight best practicesIntroduction and HDInsight best practices
Introduction and HDInsight best practices
 
Running Production CDC Ingestion Pipelines With Balaji Varadarajan and Pritam...
Running Production CDC Ingestion Pipelines With Balaji Varadarajan and Pritam...Running Production CDC Ingestion Pipelines With Balaji Varadarajan and Pritam...
Running Production CDC Ingestion Pipelines With Balaji Varadarajan and Pritam...
 
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
 
The other Apache technologies your big data solution needs!
The other Apache technologies your big data solution needs!The other Apache technologies your big data solution needs!
The other Apache technologies your big data solution needs!
 
Ch 22: Web Hosting and Internet Servers
Ch 22: Web Hosting and Internet ServersCh 22: Web Hosting and Internet Servers
Ch 22: Web Hosting and Internet Servers
 
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
Startup Case Study: Leveraging the Broad Hadoop Ecosystem to Develop World-Fi...
 
Redis tutoring
Redis tutoringRedis tutoring
Redis tutoring
 
Apache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingApache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data Processing
 
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
 
The Open Source and Cloud Part of Oracle Big Data Cloud Service for Beginners
The Open Source and Cloud Part of Oracle Big Data Cloud Service for BeginnersThe Open Source and Cloud Part of Oracle Big Data Cloud Service for Beginners
The Open Source and Cloud Part of Oracle Big Data Cloud Service for Beginners
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study Notes
 
Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...
 
Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizon
 
The other Apache Technologies your Big Data solution needs
The other Apache Technologies your Big Data solution needsThe other Apache Technologies your Big Data solution needs
The other Apache Technologies your Big Data solution needs
 
Optimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public CloudOptimizing Big Data to run in the Public Cloud
Optimizing Big Data to run in the Public Cloud
 
Apache Flink and Apache Hudi.pdf
Apache Flink and Apache Hudi.pdfApache Flink and Apache Hudi.pdf
Apache Flink and Apache Hudi.pdf
 
SQL on Hadoop
SQL on HadoopSQL on Hadoop
SQL on Hadoop
 

Mehr von HostedbyConfluent

Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
HostedbyConfluent
 
Evolution of NRT Data Ingestion Pipeline at Trendyol
Evolution of NRT Data Ingestion Pipeline at TrendyolEvolution of NRT Data Ingestion Pipeline at Trendyol
Evolution of NRT Data Ingestion Pipeline at Trendyol
HostedbyConfluent
 

Mehr von HostedbyConfluent (20)

Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
Transforming Data Streams with Kafka Connect: An Introduction to Single Messa...
 
Renaming a Kafka Topic | Kafka Summit London
Renaming a Kafka Topic | Kafka Summit LondonRenaming a Kafka Topic | Kafka Summit London
Renaming a Kafka Topic | Kafka Summit London
 
Evolution of NRT Data Ingestion Pipeline at Trendyol
Evolution of NRT Data Ingestion Pipeline at TrendyolEvolution of NRT Data Ingestion Pipeline at Trendyol
Evolution of NRT Data Ingestion Pipeline at Trendyol
 
Ensuring Kafka Service Resilience: A Dive into Health-Checking Techniques
Ensuring Kafka Service Resilience: A Dive into Health-Checking TechniquesEnsuring Kafka Service Resilience: A Dive into Health-Checking Techniques
Ensuring Kafka Service Resilience: A Dive into Health-Checking Techniques
 
Exactly-once Stream Processing with Arroyo and Kafka
Exactly-once Stream Processing with Arroyo and KafkaExactly-once Stream Processing with Arroyo and Kafka
Exactly-once Stream Processing with Arroyo and Kafka
 
Fish Plays Pokemon | Kafka Summit London
Fish Plays Pokemon | Kafka Summit LondonFish Plays Pokemon | Kafka Summit London
Fish Plays Pokemon | Kafka Summit London
 
Tiered Storage 101 | Kafla Summit London
Tiered Storage 101 | Kafla Summit LondonTiered Storage 101 | Kafla Summit London
Tiered Storage 101 | Kafla Summit London
 
Building a Self-Service Stream Processing Portal: How And Why
Building a Self-Service Stream Processing Portal: How And WhyBuilding a Self-Service Stream Processing Portal: How And Why
Building a Self-Service Stream Processing Portal: How And Why
 
From the Trenches: Improving Kafka Connect Source Connector Ingestion from 7 ...
From the Trenches: Improving Kafka Connect Source Connector Ingestion from 7 ...From the Trenches: Improving Kafka Connect Source Connector Ingestion from 7 ...
From the Trenches: Improving Kafka Connect Source Connector Ingestion from 7 ...
 
Future with Zero Down-Time: End-to-end Resiliency with Chaos Engineering and ...
Future with Zero Down-Time: End-to-end Resiliency with Chaos Engineering and ...Future with Zero Down-Time: End-to-end Resiliency with Chaos Engineering and ...
Future with Zero Down-Time: End-to-end Resiliency with Chaos Engineering and ...
 
Navigating Private Network Connectivity Options for Kafka Clusters
Navigating Private Network Connectivity Options for Kafka ClustersNavigating Private Network Connectivity Options for Kafka Clusters
Navigating Private Network Connectivity Options for Kafka Clusters
 
Apache Flink: Building a Company-wide Self-service Streaming Data Platform
Apache Flink: Building a Company-wide Self-service Streaming Data PlatformApache Flink: Building a Company-wide Self-service Streaming Data Platform
Apache Flink: Building a Company-wide Self-service Streaming Data Platform
 
Explaining How Real-Time GenAI Works in a Noisy Pub
Explaining How Real-Time GenAI Works in a Noisy PubExplaining How Real-Time GenAI Works in a Noisy Pub
Explaining How Real-Time GenAI Works in a Noisy Pub
 
TL;DR Kafka Metrics | Kafka Summit London
TL;DR Kafka Metrics | Kafka Summit LondonTL;DR Kafka Metrics | Kafka Summit London
TL;DR Kafka Metrics | Kafka Summit London
 
A Window Into Your Kafka Streams Tasks | KSL
A Window Into Your Kafka Streams Tasks | KSLA Window Into Your Kafka Streams Tasks | KSL
A Window Into Your Kafka Streams Tasks | KSL
 
Mastering Kafka Producer Configs: A Guide to Optimizing Performance
Mastering Kafka Producer Configs: A Guide to Optimizing PerformanceMastering Kafka Producer Configs: A Guide to Optimizing Performance
Mastering Kafka Producer Configs: A Guide to Optimizing Performance
 
Data Contracts Management: Schema Registry and Beyond
Data Contracts Management: Schema Registry and BeyondData Contracts Management: Schema Registry and Beyond
Data Contracts Management: Schema Registry and Beyond
 
Code-First Approach: Crafting Efficient Flink Apps
Code-First Approach: Crafting Efficient Flink AppsCode-First Approach: Crafting Efficient Flink Apps
Code-First Approach: Crafting Efficient Flink Apps
 
Debezium vs. the World: An Overview of the CDC Ecosystem
Debezium vs. the World: An Overview of the CDC EcosystemDebezium vs. the World: An Overview of the CDC Ecosystem
Debezium vs. the World: An Overview of the CDC Ecosystem
 
Beyond Tiered Storage: Serverless Kafka with No Local Disks
Beyond Tiered Storage: Serverless Kafka with No Local DisksBeyond Tiered Storage: Serverless Kafka with No Local Disks
Beyond Tiered Storage: Serverless Kafka with No Local Disks
 

Kürzlich hochgeladen

Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
vu2urc
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Service
giselly40
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
Enterprise Knowledge
 

Kürzlich hochgeladen (20)

08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)
 
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdfUnderstanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Service
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
 
Handwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsHandwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed texts
 
Evaluating the top large language models.pdf
Evaluating the top large language models.pdfEvaluating the top large language models.pdf
Evaluating the top large language models.pdf
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 

Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apache Software Foundation

  • 1. Streaming Data Lakes Using Kafka Connect +Apache Hudi Balaji Varadarajan, Vinoth Chandar
  • 2. Speakers Vinoth Chandar PMC Chair/Creator of Hudi Sr.Staff Eng @ Uber (Data Infra/Platforms, Networking) Principal Eng @ Confluent (ksqlDB, Kafka/Streams) Staff Eng @ Linkedin (Voldemort, DDS) Sr Eng @ Oracle (CDC/Goldengate/XStream) Balaji Varadarajan PMC Member, Apache Hudi Sr. Staff Eng @ Robinhood, Data Infra Tech Lead @Uber, Data Platform Staff Engineer @Linkedin, Databus CDC
  • 3. Agenda 1) Background 2) Hudi 101 3) Hudi’s Spark Writers (existing) 4) Kafka Connect Sink (new) 5) Onwards
  • 5. Data Lakes are now essential Architectural Pattern for Analytical Data ❏ Data Lake != Spark, Flink ❏ Data Lake != Files on S3 ❏ Raw data (OLTP schema) ❏ Derived Data (OLAP/BI, ML schema) Open Storage + Scalable Compute ❏ Avoid data lock-in, Open formats (data + metadata) ❏ Efficient, Universal (Analytics, Data Science) Lot of exciting progress ❏ Lakehouse = Lake + Warehouse ❏ Data meshes on Lakes => Need for streams Source: https://martinfowler.com/bliki/images/dataLake/context.png
  • 6. Event Streams are the new norm Events come in many flavors Database change Events ❏ High fidelity, High value, update/deletes ❏ E.g: Debezium changelogs into Kafka Application/Service business events ❏ High volume, Immutable or Deltas, ❏ E.g: Emit Uber app events, emit changes from IoT sensors SaaS Data Sources ❏ Lower volume, mutable ❏ E.g: polling Github events API
  • 8. Why not just Connect File Sinks? Queries DFS/Cloud Storage Data Lake?? Files Kafka Cluster Kafka Connect Sinks (S3/HDFS)
  • 9. Challenges Working at the file abstraction level is painful ❏ Transactional, Concurrency Control ❏ Updates subset of data, indexing for faster access Scalability, Operational Overhead ❏ Writing columnar files is resource intensive ❏ Partitioned data increases memory overhead Lack of management ❏ Control file sizes, Deletes for GDPR/Compliance ❏ Re-align storage for better query performance
  • 10. Apache Hudi Transactional Writes, MVCC/OCC ❏ Work with tables and records ❏ Automatic compaction, clustering, sizing First class support for Updates, Deletes ❏ Record level Update/Deletes inspired by stream processors CDC Streams From Lake Storage ❏ Storage Layout optimized for incremental fetches ❏ Hudi’s unique contribution in the space
  • 12. Stream processing + Batch data The Incremental Stack + Intelligent, Incremental + Fast, Efficient + Scans, Columnar formats + Scalable Compute https://www.oreilly.com/content/ubers-case-for- incremental-processing-on-hadoop/; 2016
  • 13. The Hudi Stack ❏ Complete “data” lake platform ❏ Tightly integrated, Self managing ❏ Write using Spark, Flink ❏ Query using Spark, Flink, Hive, Presto, Trino, Impala, AWS Athena/Redshift, Aliyun DLA etc ❏ Out-of-box tools/services for data ops http://hudi.apache.org/blog/2021/07/21/st reaming-data-lake-platform
  • 15. ❏ Powers arguably the largest transactional data lake on the planet @ Uber ❏ (Database CDC) Robinhood’s near-realtime data lake ❏ (ML Feature stores) @ Logical Clocks ❏ (Event Deletions/De-Duping) @ Moveworks ❏ Many more companies, pre-installed by 5 major cloud providers 1000+ Slack members 150+ Contributors 1000+ GH Engagers ~10-20 PRs/week 20+ Committers 10+ PMCs The Community
  • 17. Event Streams DFS/Cloud Storage Tables Pull using Spark Kafka De-Dupe Indexing Txn DeltaStreamer Utility, Spark Streaming Cluster Optimize Compact Apply Pull Cleaning
  • 18. Current Kafka to Hudi Options - Ingest streaming data to Data Lake - Raw Tables - Current Solutions through Spark: - Hudi DeltaStreamer - Spark Structured Streaming Kafka Cluster Hudi DeltaStreamer Spark Structured Streaming DFS/Cloud Storage Tables Apply
  • 19. Structured Streaming Sink // Read data from stream Dataset<Row> streamingInput = spark.readStream()... // Write to Hudi in a streaming fashion DataStreamWriter<Row> writer = streamingInput.writeStream() .format("org.apache.hudi") .option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType) .option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "timestamp") .option(HoodieWriteConfig.TABLE_NAME.key(), tableName) .option("checkpointLocation", checkpointLocation) .outputMode(OutputMode.Append()); String tablePath = “s3://…." // Schedule the job StreamingQuery query = ... writer.trigger(Trigger.ProcessingTime(500)).start(tablePath); query.awaitTermination(streamingDurationInMs);
  • 20. DeltaStreamer Utility ❏ Fully Managed Ingestion and ETL service ❏ Integration with various Streaming and batch sources ❏ Table State & Checkpoints transactionally consistent ❏ Pluggable Transformations for ETL use cases.
  • 21. DeltaStreamer Example spark-submit --master yarn --packages org.apache.hudi:hudi-utilities-bundle_2.12:0.8.0 --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer --conf spark.scheduler.mode=FAIR --conf spark.task.maxFailures=5 ... --enable-sync --hoodie-conf auto.offset.reset=latest --hoodie-conf hoodie.avro.schema.validate=true …. --table-type MERGE_ON_READ --source-class org.apache.hudi.utilities.sources.AvroKafkaSource --schemaprovider-class org.apache.hudi.utilities.schema.SchemaRegistryProvider --props /path/job.properties --transformer-class com.some.someTransformer --continuous ← Enables async compaction, clustering & cleaning along with streaming writes Streaming Data Lake without writing any code!
  • 22. Case Study: Robinhood Data Lake Master RDS Replica RDS Table Topic DeltaStreamer (Live) DeltaStreamer (Bootstrap) DATA LAKE (s3://xxx/… Update schema and partition Write incremental data and checkpoint offsets
  • 23. Case Study: Robinhood Data Lake ❏ 1000s of CDC based Streaming ingest pipelines supported by Apache Hudi DeltaStreamer. ❏ Data Lake freshness Latency down to 5-15 mins from hours. ❏ Powers critical dashboards and use-cases
  • 24. End-to-End Streaming Data Lake ❏ Data Lake has both raw tables and derived tables built through ETLs. ❏ Streaming Data-lake - Needs streaming semantics supported for both kinds of tables. ❏ The Missing Primitive : Derived Tables need Changelog view of the upstream dataset -> Apache Hudi Incremental Read to rescue
  • 25. The Big Picture Pull Database Event Streams Apps/ Service s External Sources CDC Push Streaming Data Lake Raw Tables DeltaStreamer Spark Streaming Hudi Change log Derived Tables DeltaStreamer Spark Streaming
  • 26. Connect Hudi Sink Kafkaesque, Commit protocol, Transactional
  • 27. Motivations Integration with Kafka Connect ❏ Separation of concerns (writing vs optimization/management) ❏ Streamline operationally, just one framework for ingesting ❏ Less need for Spark expertise Faster data ❏ Amortize startup costs (containers, queue delays) ❏ Commit frequently i.e every 1 minute (every N secs in near future) ❏ E.g avro records in Kafka log to Hudi’s log format
  • 28. Putting it all together Event Streams DFS/Cloud Storage Tables Kafka De-Dupe Indexing Txn Hudi Connect Sink (Writing) Commit Pull Compact Cluster Hudi’s Table Services (Optimization, management) Clean Deletes
  • 29. Design Challenges Determining Transaction Boundaries ❏ No co-ordination via driver process like Spark/Flink ❏ Workers doing their own commits => horrible concurrency bottlenecks Connect APIs cannot express DAGs ❏ Meant to be simple `putRecords()`/`preCommit()` ❏ Indexing, De-duplication, Storage optimization all shuffle data
  • 30. Design Overview Central Transaction Co-ordination ❏ Use Kafka to elect co- ordinator. ❏ Runs in one of the workers Kafka as control channel ❏ Consume from latest control topic offsets https://cwiki.apache.org/confluence/display/HUDI/RFC-32+Kafka+Connect+Sink+for+Hudi
  • 31. Design Overview Transaction Coordinator ❏ Daemon thread on owner of partition 0 ❏ Sends commands to participants Embedded Hudi Java Writer ❏ Lands data into set of file groups, mapped to a partition ❏ Hudi’s commit fencing guards from failures/partial writes
  • 32. Co-ordinator State Machine Paxos-like two phase commit ❏ Co-ordinator process to start, end commits ❏ Safety > liveness, abort after timeout Participants “pause” at each commit boundary ❏ Return latest write offsets to co-ordinator ❏ Resume again on start of next commit
  • 33. Example Sink Configuration # hudi table properties target.base.path target.table.name target.database.name schemaprovider.class partition.field.name hoodie.table.base.file.format Pre-release, subject to change. Refer to official Hudi docs, for actual config names. # controller properties control.topic.name coordinator.writestatus.timeout write.retry.timeout
  • 34. Choosing Right Delta Streamer Connect Sink Provides full set of Hudi features Insert only for now, indexes/updates coming as enhancements Offers better elasticity for merging/writing columnar data i.e copy-on-write tables Great impedance match with Kafka, for landing avro/row-oriented data i.e merge-on- read tables Data freshness of several minutes, if not running in continuous mode Approach ~1 min freshness Need experience with Spark/Flink Operate all data ingestion in a single framework.
  • 36. Kafka + Hudi Support for mutable, keyed updates/deletes ❏ Need to implement a new index ala Flink writer ❏ preCombine, buffering/batching What if : Back Kafka’s tiered storage using Hudi ❏ Map offsets to Hudi commit_seq_no ❏ Columnar reads for historical/catch-up reads
  • 37. Engage With Our Community User Docs : https://hudi.apache.org Technical Wiki : https://cwiki.apache.org/confluence/display/HUDI Github : https://github.com/apache/hudi/ Twitter : https://twitter.com/apachehudi Mailing list(s) : dev-subscribe@hudi.apache.org (send an empty email to subscribe) dev@hudi.apache.org (actual mailing list) Slack : https://join.slack.com/t/apache-hudi/signup