SlideShare una empresa de Scribd logo
1 de 38
Descargar para leer sin conexión
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!

Más contenido relacionado

La actualidad más candente

Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3DataWorks Summit
 
Free Training: How to Build a Lakehouse
Free Training: How to Build a LakehouseFree Training: How to Build a Lakehouse
Free Training: How to Build a LakehouseDatabricks
 
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 DebeziumTathastu.ai
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Flink Forward
 
Apache Hudi: The Path Forward
Apache Hudi: The Path ForwardApache Hudi: The Path Forward
Apache Hudi: The Path ForwardAlluxio, Inc.
 
Hadoop Security Architecture
Hadoop Security ArchitectureHadoop Security Architecture
Hadoop Security ArchitectureOwen O'Malley
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of HadoopDatabricks
 
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)Ryan Blue
 
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 CoversScyllaDB
 
Building Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeBuilding Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeFlink Forward
 
A Reference Architecture for ETL 2.0
A Reference Architecture for ETL 2.0 A Reference Architecture for ETL 2.0
A Reference Architecture for ETL 2.0 DataWorks Summit
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Data Engineer's Lunch #83: Strategies for Migration to Apache Iceberg
Data Engineer's Lunch #83: Strategies for Migration to Apache IcebergData Engineer's Lunch #83: Strategies for Migration to Apache Iceberg
Data Engineer's Lunch #83: Strategies for Migration to Apache IcebergAnant Corporation
 
Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing DataWorks Summit
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...StreamNative
 
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...DataScienceConferenc1
 
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...StreamNative
 
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 LakeDatabricks
 

La actualidad más candente (20)

Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3
 
Free Training: How to Build a Lakehouse
Free Training: How to Build a LakehouseFree Training: How to Build a Lakehouse
Free Training: How to Build a Lakehouse
 
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
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
 
Apache Hudi: The Path Forward
Apache Hudi: The Path ForwardApache Hudi: The Path Forward
Apache Hudi: The Path Forward
 
Hadoop Security Architecture
Hadoop Security ArchitectureHadoop Security Architecture
Hadoop Security Architecture
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
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 Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 
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
 
Building Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeBuilding Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta Lake
 
A Reference Architecture for ETL 2.0
A Reference Architecture for ETL 2.0 A Reference Architecture for ETL 2.0
A Reference Architecture for ETL 2.0
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Data Engineer's Lunch #83: Strategies for Migration to Apache Iceberg
Data Engineer's Lunch #83: Strategies for Migration to Apache IcebergData Engineer's Lunch #83: Strategies for Migration to Apache Iceberg
Data Engineer's Lunch #83: Strategies for Migration to Apache Iceberg
 
Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing
 
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 ...
 
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...
[DSC Europe 22] Lakehouse architecture with Delta Lake and Databricks - Draga...
 
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...
 
From Data Warehouse to Lakehouse
From Data Warehouse to LakehouseFrom Data Warehouse to Lakehouse
From Data Warehouse to Lakehouse
 
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
 

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

SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...Chester Chen
 
[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/HudiVinoth Chandar
 
Introduction and HDInsight best practices
Introduction and HDInsight best practicesIntroduction and HDInsight best practices
Introduction and HDInsight best practicesAshish Thapliyal
 
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...HostedbyConfluent
 
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...Data Con LA
 
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!gagravarr
 
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 Serverswebhostingguy
 
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
 
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 ProcessingDataWorks Summit
 
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...Lucidworks
 
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 BeginnersEdelweiss Kammermann
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study NotesRichard Kuo
 
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 ...Alluxio, Inc.
 
Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizonArtem Ervits
 
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 needsgagravarr
 
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 CloudQubole
 
Apache Flink and Apache Hudi.pdf
Apache Flink and Apache Hudi.pdfApache Flink and Apache Hudi.pdf
Apache Flink and Apache Hudi.pdfdogma28
 
SQL on Hadoop
SQL on HadoopSQL on Hadoop
SQL on Hadoopnvvrajesh
 
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)Data processing at the speed of 100 Gbps@Apache Crail (Incubating)
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)DataWorks Summit
 

Similar a Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apache Software Foundation (20)

SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
SF Big Analytics 20190612: Building highly efficient data lakes using Apache ...
 
[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
 
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)Data processing at the speed of 100 Gbps@Apache Crail (Incubating)
Data processing at the speed of 100 Gbps@Apache Crail (Incubating)
 

Más de 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
 
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 LondonHostedbyConfluent
 
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 TrendyolHostedbyConfluent
 
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 TechniquesHostedbyConfluent
 
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 KafkaHostedbyConfluent
 
Fish Plays Pokemon | Kafka Summit London
Fish Plays Pokemon | Kafka Summit LondonFish Plays Pokemon | Kafka Summit London
Fish Plays Pokemon | Kafka Summit LondonHostedbyConfluent
 
Tiered Storage 101 | Kafla Summit London
Tiered Storage 101 | Kafla Summit LondonTiered Storage 101 | Kafla Summit London
Tiered Storage 101 | Kafla Summit LondonHostedbyConfluent
 
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 WhyHostedbyConfluent
 
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 ...HostedbyConfluent
 
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 ...HostedbyConfluent
 
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 ClustersHostedbyConfluent
 
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 PlatformHostedbyConfluent
 
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 PubHostedbyConfluent
 
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 LondonHostedbyConfluent
 
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 | KSLHostedbyConfluent
 
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 PerformanceHostedbyConfluent
 
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 BeyondHostedbyConfluent
 
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 AppsHostedbyConfluent
 
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 EcosystemHostedbyConfluent
 
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 DisksHostedbyConfluent
 

Más de 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
 

Último

What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?Antenna Manufacturer Coco
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfsudhanshuwaghmare1
 
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Igalia
 
Advantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessAdvantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessPixlogix Infotech
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEarley Information Science
 
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 SolutionsEnterprise Knowledge
 
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...Neo4j
 
Real Time Object Detection Using Open CV
Real Time Object Detection Using Open CVReal Time Object Detection Using Open CV
Real Time Object Detection Using Open CVKhem
 
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...Enterprise Knowledge
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
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 MenDelhi Call girls
 
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...Drew Madelung
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
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 slidevu2urc
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking MenDelhi Call girls
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreternaman860154
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfEnterprise Knowledge
 
How to convert PDF to text with Nanonets
How to convert PDF to text with NanonetsHow to convert PDF to text with Nanonets
How to convert PDF to text with Nanonetsnaman860154
 
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 Servicegiselly40
 

Último (20)

What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdf
 
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
 
Advantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessAdvantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your Business
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
 
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
 
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
 
Real Time Object Detection Using Open CV
Real Time Object Detection Using Open CVReal Time Object Detection Using Open CV
Real Time Object Detection Using Open CV
 
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...
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
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
 
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...
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
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
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreter
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
 
How to convert PDF to text with Nanonets
How to convert PDF to text with NanonetsHow to convert PDF to text with Nanonets
How to convert PDF to text with Nanonets
 
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
 

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