SlideShare a Scribd company logo
1 of 30
Download to read offline
Simplify CDC pipeline with Spark
Streaming SQL and Delta Lake
Jun Song@windpiger
songjun.sj@alibaba-inc.com
June 24, 2020
About Me
• Staff Engineer from Alibaba Cloud E-MapReduce Product Team
• Spark contributor focused on SparkSQL
• HiveOnDelta contributor(https://github.com/delta-io/connectors)
Agenda
• What is CDC
• CDC solution using Spark
Streaming SQL & Delta Lake
• Future Works
What is CDC
Change Data Capture
Target
Storage
AdHoc/
ETL
Collect Change Sets & Merge
Analytics
CDC
OLTP database
Data Warehouse
Data Lake
…
▪ load pressure on source database
▪ high latency batch job(hourly/daily/…)
▪ can not handle delete rows
▪ can not handle schema change
DrawbacksUsing Sqoop(Batch Mode)
Change Data Capture
sqoop
--incremental lastmodified
--last-value '2028/01/01 13:00:00’
...
sqoop merge
--new-data newer
--onto older
--merge-key id
…
▪ heavy servers & operational
support(Kudu&HBase)
▪ HBase can not support high-throughput
analytics
▪ complex merge implement by java/scala code
▪ can not handle schema change
DrawbacksUsing binlog(Streaming Mode)
Change Data Capture
binlog
scala/java
CDC solution using Spark Streaming SQL & Delta
Spark Streaming SQL
SparkCore
SparkSQL
Structured Streaming
Spark Streaming SQL
https://www.alibabacloud.com/help/doc-detail/124684.htm
SQL is a Standard Declarative Language, which can simplify real-time analytics
▪ DDL
CREATE TABLE、CREATE TABLE AS SELECT、CREATE SCAN、CREATE STREAM
▪ DML
INSERT INTO、MERGE INTO
▪ SELECT
SELECT FROM、WHERE、GROUP BY 、JOIN、UNION ALL
▪ UDF
TUMBLING、HOPPING、DELAY、SparkSQL UDF
▪ Data Source
Delta、Kafka、HBase、JDBC、Druid、Redis、Kudu、Alibaba Cloud(Loghub、Tablestore、DataHub)
Design Doc: https://docs.google.com/document/d/19degwnIIcuMSELv6BQ_1VQI5AIVcvGeqOm5xE2-aRA0/edit
Spark Streaming SQL
CREATE TABLE kafka_test
USING kafka
OPTIONS(
kafka.bootstrap.servers='',
subscribe='test')
batch
streaming
CREATE SCAN kafka_test_batch_scan
ON kafka_test
USING batch
CREATE SCAN kafka_test_batch_scan
ON kafka_test
USING stream
OPTIONS(
maxOffsetsPerTrigger='100000'
)
SELECT count(*)
FROM kafka_test_batch_scan
?
CREATE SCAN
CREATE SCAN tbName_alias
ON tbName
USING queryType
OPTIONS (propertyName=propertyValue[,propertyName=propertyValue]*)
Spark Streaming SQL
CREATE SCAN kafka_test_batch_scan
ON kafka_test
USING stream
OPTIONS(
maxOffsetsPerTrigger='100000'
)
CREATE STREAM
CREATE STREAM kafka_test_stream_job
OPTIONS(
checkpointLocation='/tmp/spark',
outputMode='Append',
triggerType='ProcessingTime'
triggerIntervalMs='3000')
INSERT INTO target_tbl
SELECT * FROM kafka_test_batch_scan
WHERE units > 1000;
CREATE STREAM queryName
OPTIONS (propertyName=propertyValue[,propertyName=propertyValue]*)
INSERT INTO tbName
queryStatement;
Spark Streaming SQL
MERGE INTO
mergeInto
: MERGE INTO target=tableIdentifier tableAlias
USING (source=tableIdentifier (timeTravel)? | '(' subquery = query ')') tableAlias
mergeCondition?
matchedClauses*
notMatchedClause?
MERGE INTO target_table t
USING source_table s
ON s.id = t.id
WHEN MATCHED AND s.opType = 'delete' THEN DELETE
WHEN MATCHED AND s.opTye = 'update' THEN UPDATE SET id = s.id, name = s.name
WHEN NOT MATCHED AND s.opType = 'insert' THEN INSERT (key, value) VALUES (key, value)
Spark Streaming SQL
DELAY / TUMBLING / HOPPING
WHERE delay(colName) < 'duration'withWatermark("colName", "duration")
SELECT avg(inv_quantity_on_hand) qoh
FROM kafka_inventory
WHERE delay(inv_data_time) < '2 minutes'
GROUP BY TUMBLING (inv_data_time, interval 1 minute)
Delta Lake
Streaming/Batch
Streaming/Batch
ACID Transactions
Metadata Management
Unified Batch&Streaming Schema Enforcement&Evolution
Update&Delete&Merge
Time Travel
Parquet
Key Feature
Delta Lake
Improvement
Delta Lake
SparkSQL Spark Streaming SQL
Update/Delete/
Optimize/Vacuum
DDL/DML
Hive/
Presto
HDFS Alibaba OSS S3 …
Delta Lake
Improvement
CREATE EXTERNAL TABLE delta_tbl(a string, b int)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
STORED AS INPUTFORMAT 'io.delta.hive.DeltaInputFormat'
OUTPUTFORMAT 'io.delta.hive.DeltaOutputFormat'
LOCATION 'oss://testbucket/delta/events'
Spark Streaming SQL
binlog
▪ no extra operational support for delta
▪ no load pressure on source database
▪ merge implement easily by SQL
▪ realtime low-latency(minute)
CDC solution using Spark Streaming SQL & Delta Lake
Spark Streaming SQL
binlog
CREATE SCAN cdctest_incremental_scan ON kafka_cdctest
USING STREAM
OPTIONS(
startingOffsets='earliest',
maxOffsetsPerTrigger='100000',
failOnDataLoss=false
);
CREATE STREAM cdctest_job
OPTIONS(checkpointLocation='/delta/cdctest_checkpoint_oss')
MERGE INTO delta_cdctest_oss as target
USING (
SELECT
// binlog parser
…
FROM cdctest_incremental_scan
)
ON target.id = source.before_id
WHEN MATCHED AND source.recordType='UPDATE' THEN
UPDATE SET …
WHEN MATCHED AND source.recordType='DELETE' THEN
DELETE
WHEN NOT MATCHED AND source.recordType='INSERT' THEN
INSERT …
CREATE TABLE kafka_cdctest USING KAFKA …
CREATE TABLE delta_cdctest_oss USING DELTA …
streaming-sql --master yarn --use-emr-datasource -f cdc_oss.sql
1
2
3
4
Delta Table
batch-batch- batch- …batch-
DeltaTable
.merge
DeltaTable
.merge
DeltaTable
.merge
DeltaTable
.merge
Spark Streaming SQL MERGE INTO
CDC solution using Spark Streaming SQL & Delta Lake
Long Running Stability Improvement
How to handle small files?
▪ increase batch interval(minutes)
▪ compcation(change data layout, not change data)
▪ adaptive execution mode
Long Running Stability Improvement
How to handle small files? - Scheduled Compaction
Delta Table
batch-batch- batch- …
batch-
DeltaTable
.merge
DeltaTable
.merge
DeltaTable
.merge
DeltaTable
.merge
CompactionCompaction
scheduled job hourly/daily/…
OPTIMIZE <tbl> [WHERE where_clause]
Long Running Stability Improvement
Problem: Streaming Job Failed when doing a compaction
batch-
Delta Table
Compaction do transaction commit
merge
Transaction conflict check
read
How to handle small files? - Scheduled Compaction
Long Running Stability Improvement
Problem: Streaming Job Failed when doing a compaction
binlog in batch
streaming job
status
Improvement
only insert Succeed Fix one bug: https://github.com/delta-io/delta/issues/326
including
delete/update Failed streaming job retry this batch
How to handle small files? - Scheduled Compaction
Long Running Stability Improvement
How to handle small files? - Auto Compaction
Delta Table
batch-batch- …batch-
merge
compaction compaction
merge merge
sequential execution, no conflict
select files which file_size
> COMPACT_FILE_SIZE
flies
number >
TRIGGER_FILE_COU
NT
do compaction
Yes
No
continue streaming
Strategy
Long Running Stability Improvement
How to handle small files? - Adaptive Execution
batch
binlog
target
delta
target
changed files
Join
rewrite
all changed files
Join
spark.sql.adaptive.enabled -> true
Adaptive Excetion can auto merge small partitions,
to decrease the number of reducers, then decrease
the number of output files.
Long Running Stability Improvement
Performance issue
batch-batch- batch- …batch-
size of target delta table
merge performance
Runtime Filter https://issues.apache.org/jira/browse/SPARK-27227
batch
binlog
target
delta
target
changed files
Join
filter
Future Works
Future
▪ auto schema change detected
▪ long running stable performance(read on merge)
▪ simplify user experience by SYNC grammar
SYNC kafka_binlog_tbl
TO delta_tbl
OPTIONS(
type='debezium.mysql'
)
Feedback
Your feedback is important to us.
Don’t forget to rate and
review the sessions.

More Related Content

What's hot

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 TechnologyMatei Zaharia
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in DeltaDatabricks
 
Delta Lake OSS: Create reliable and performant Data Lake by Quentin Ambard
Delta Lake OSS: Create reliable and performant Data Lake by Quentin AmbardDelta Lake OSS: Create reliable and performant Data Lake by Quentin Ambard
Delta Lake OSS: Create reliable and performant Data Lake by Quentin AmbardParis Data Engineers !
 
Spark with Delta Lake
Spark with Delta LakeSpark with Delta Lake
Spark with Delta LakeKnoldus Inc.
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Databricks
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingDatabricks
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark Summit
 
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021StreamNative
 
Delta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDelta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDatabricks
 
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Databricks
 
Solving Enterprise Data Challenges with Apache Arrow
Solving Enterprise Data Challenges with Apache ArrowSolving Enterprise Data Challenges with Apache Arrow
Solving Enterprise Data Challenges with Apache ArrowWes McKinney
 
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
 
Building an open data platform with apache iceberg
Building an open data platform with apache icebergBuilding an open data platform with apache iceberg
Building an open data platform with apache icebergAlluxio, Inc.
 
Modularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkModularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkDatabricks
 
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 ...Databricks
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesDatabricks
 
Streaming SQL for Data Engineers: The Next Big Thing?
Streaming SQL for Data Engineers: The Next Big Thing?Streaming SQL for Data Engineers: The Next Big Thing?
Streaming SQL for Data Engineers: The Next Big Thing?Yaroslav Tkachenko
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveDataWorks Summit
 

What's hot (20)

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
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
 
Delta Lake OSS: Create reliable and performant Data Lake by Quentin Ambard
Delta Lake OSS: Create reliable and performant Data Lake by Quentin AmbardDelta Lake OSS: Create reliable and performant Data Lake by Quentin Ambard
Delta Lake OSS: Create reliable and performant Data Lake by Quentin Ambard
 
Spark with Delta Lake
Spark with Delta LakeSpark with Delta Lake
Spark with Delta Lake
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured Streaming
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
 
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021
Trino: A Ludicrously Fast Query Engine - Pulsar Summit NA 2021
 
Delta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDelta from a Data Engineer's Perspective
Delta from a Data Engineer's Perspective
 
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
 
Solving Enterprise Data Challenges with Apache Arrow
Solving Enterprise Data Challenges with Apache ArrowSolving Enterprise Data Challenges with Apache Arrow
Solving Enterprise Data Challenges with Apache Arrow
 
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...
 
Building an open data platform with apache iceberg
Building an open data platform with apache icebergBuilding an open data platform with apache iceberg
Building an open data platform with apache iceberg
 
Modularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkModularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache Spark
 
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 ...
 
Unified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache FlinkUnified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache Flink
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization Opportunities
 
Streaming SQL for Data Engineers: The Next Big Thing?
Streaming SQL for Data Engineers: The Next Big Thing?Streaming SQL for Data Engineers: The Next Big Thing?
Streaming SQL for Data Engineers: The Next Big Thing?
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
 

Similar to Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake

Self-serve analytics journey at Celtra: Snowflake, Spark, and Databricks
Self-serve analytics journey at Celtra: Snowflake, Spark, and DatabricksSelf-serve analytics journey at Celtra: Snowflake, Spark, and Databricks
Self-serve analytics journey at Celtra: Snowflake, Spark, and DatabricksGrega Kespret
 
In-memory ColumnStore Index
In-memory ColumnStore IndexIn-memory ColumnStore Index
In-memory ColumnStore IndexSolidQ
 
iceberg introduction.pptx
iceberg introduction.pptxiceberg introduction.pptx
iceberg introduction.pptxDori Waldman
 
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...StreamNative
 
High Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureHigh Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureDataStax Academy
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeDatabricks
 
Denver SQL Saturday The Next Frontier
Denver SQL Saturday The Next FrontierDenver SQL Saturday The Next Frontier
Denver SQL Saturday The Next FrontierKellyn Pot'Vin-Gorman
 
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...djkucera
 
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase DataWorks Summit
 
ETL and pivoting in spark
ETL and pivoting in sparkETL and pivoting in spark
ETL and pivoting in sparkSubhasish Guha
 
ETL and pivoting in spark
ETL and pivoting in sparkETL and pivoting in spark
ETL and pivoting in sparkSubhasish Guha
 
Manchester Hadoop Meetup: Spark Cassandra Integration
Manchester Hadoop Meetup: Spark Cassandra IntegrationManchester Hadoop Meetup: Spark Cassandra Integration
Manchester Hadoop Meetup: Spark Cassandra IntegrationChristopher Batey
 
Operating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionOperating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionDatabricks
 
Building a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkBuilding a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkEvan Chan
 
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?Jim Czuprynski
 
User Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBUser Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBKai Sasaki
 
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...Accumulo Summit
 

Similar to Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake (20)

Self-serve analytics journey at Celtra: Snowflake, Spark, and Databricks
Self-serve analytics journey at Celtra: Snowflake, Spark, and DatabricksSelf-serve analytics journey at Celtra: Snowflake, Spark, and Databricks
Self-serve analytics journey at Celtra: Snowflake, Spark, and Databricks
 
In-memory ColumnStore Index
In-memory ColumnStore IndexIn-memory ColumnStore Index
In-memory ColumnStore Index
 
Copy Data Management for the DBA
Copy Data Management for the DBACopy Data Management for the DBA
Copy Data Management for the DBA
 
iceberg introduction.pptx
iceberg introduction.pptxiceberg introduction.pptx
iceberg introduction.pptx
 
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
 
High Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & AzureHigh Throughput Analytics with Cassandra & Azure
High Throughput Analytics with Cassandra & Azure
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 
Denver SQL Saturday The Next Frontier
Denver SQL Saturday The Next FrontierDenver SQL Saturday The Next Frontier
Denver SQL Saturday The Next Frontier
 
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...
Collaborate 2009 - Migrating a Data Warehouse from Microsoft SQL Server to Or...
 
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase
PayPal merchant ecosystem using Apache Spark, Hive, Druid, and HBase
 
OOW13 Exadata and ODI with Parallel
OOW13 Exadata and ODI with ParallelOOW13 Exadata and ODI with Parallel
OOW13 Exadata and ODI with Parallel
 
ETL and pivoting in spark
ETL and pivoting in sparkETL and pivoting in spark
ETL and pivoting in spark
 
ETL and pivoting in spark
ETL and pivoting in sparkETL and pivoting in spark
ETL and pivoting in spark
 
Manchester Hadoop Meetup: Spark Cassandra Integration
Manchester Hadoop Meetup: Spark Cassandra IntegrationManchester Hadoop Meetup: Spark Cassandra Integration
Manchester Hadoop Meetup: Spark Cassandra Integration
 
It Depends
It DependsIt Depends
It Depends
 
Operating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionOperating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in Production
 
Building a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkBuilding a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and Spark
 
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?
Autonomous Transaction Processing (ATP): In Heavy Traffic, Why Drive Stick?
 
User Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDBUser Defined Partitioning on PlazmaDB
User Defined Partitioning on PlazmaDB
 
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...
Accumulo Summit 2015: Building Aggregation Systems on Accumulo [Leveraging Ac...
 

More from Databricks

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDatabricks
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Databricks
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Databricks
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Databricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
 
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
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDatabricks
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceDatabricks
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringDatabricks
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixDatabricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationDatabricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchDatabricks
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesDatabricks
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesDatabricks
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsDatabricks
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkDatabricks
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkDatabricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesDatabricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkDatabricks
 
Machine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionDatabricks
 

More from Databricks (20)

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

Recently uploaded

Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxolyaivanovalion
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxEmmanuel Dauda
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
Ukraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSUkraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSAishani27
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一ffjhghh
 
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptxEMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptxthyngster
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Serviceranjana rawat
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Callshivangimorya083
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxfirstjob4
 
Mature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxMature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxolyaivanovalion
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiSuhani Kapoor
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFxolyaivanovalion
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionfulawalesam
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAroojKhan71
 

Recently uploaded (20)

Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptx
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptx
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
Ukraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSUkraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICS
 
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
 
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptxEMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM  TRACKING WITH GOOGLE ANALYTICS.pptx
EMERCE - 2024 - AMSTERDAM - CROSS-PLATFORM TRACKING WITH GOOGLE ANALYTICS.pptx
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptx
 
Mature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxMature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptx
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interaction
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
 

Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake

  • 1.
  • 2. Simplify CDC pipeline with Spark Streaming SQL and Delta Lake Jun Song@windpiger songjun.sj@alibaba-inc.com June 24, 2020
  • 3. About Me • Staff Engineer from Alibaba Cloud E-MapReduce Product Team • Spark contributor focused on SparkSQL • HiveOnDelta contributor(https://github.com/delta-io/connectors)
  • 4. Agenda • What is CDC • CDC solution using Spark Streaming SQL & Delta Lake • Future Works
  • 6. Change Data Capture Target Storage AdHoc/ ETL Collect Change Sets & Merge Analytics CDC OLTP database Data Warehouse Data Lake …
  • 7. ▪ load pressure on source database ▪ high latency batch job(hourly/daily/…) ▪ can not handle delete rows ▪ can not handle schema change DrawbacksUsing Sqoop(Batch Mode) Change Data Capture sqoop --incremental lastmodified --last-value '2028/01/01 13:00:00’ ... sqoop merge --new-data newer --onto older --merge-key id …
  • 8. ▪ heavy servers & operational support(Kudu&HBase) ▪ HBase can not support high-throughput analytics ▪ complex merge implement by java/scala code ▪ can not handle schema change DrawbacksUsing binlog(Streaming Mode) Change Data Capture binlog scala/java
  • 9. CDC solution using Spark Streaming SQL & Delta
  • 10. Spark Streaming SQL SparkCore SparkSQL Structured Streaming Spark Streaming SQL https://www.alibabacloud.com/help/doc-detail/124684.htm SQL is a Standard Declarative Language, which can simplify real-time analytics ▪ DDL CREATE TABLE、CREATE TABLE AS SELECT、CREATE SCAN、CREATE STREAM ▪ DML INSERT INTO、MERGE INTO ▪ SELECT SELECT FROM、WHERE、GROUP BY 、JOIN、UNION ALL ▪ UDF TUMBLING、HOPPING、DELAY、SparkSQL UDF ▪ Data Source Delta、Kafka、HBase、JDBC、Druid、Redis、Kudu、Alibaba Cloud(Loghub、Tablestore、DataHub) Design Doc: https://docs.google.com/document/d/19degwnIIcuMSELv6BQ_1VQI5AIVcvGeqOm5xE2-aRA0/edit
  • 11. Spark Streaming SQL CREATE TABLE kafka_test USING kafka OPTIONS( kafka.bootstrap.servers='', subscribe='test') batch streaming CREATE SCAN kafka_test_batch_scan ON kafka_test USING batch CREATE SCAN kafka_test_batch_scan ON kafka_test USING stream OPTIONS( maxOffsetsPerTrigger='100000' ) SELECT count(*) FROM kafka_test_batch_scan ? CREATE SCAN CREATE SCAN tbName_alias ON tbName USING queryType OPTIONS (propertyName=propertyValue[,propertyName=propertyValue]*)
  • 12. Spark Streaming SQL CREATE SCAN kafka_test_batch_scan ON kafka_test USING stream OPTIONS( maxOffsetsPerTrigger='100000' ) CREATE STREAM CREATE STREAM kafka_test_stream_job OPTIONS( checkpointLocation='/tmp/spark', outputMode='Append', triggerType='ProcessingTime' triggerIntervalMs='3000') INSERT INTO target_tbl SELECT * FROM kafka_test_batch_scan WHERE units > 1000; CREATE STREAM queryName OPTIONS (propertyName=propertyValue[,propertyName=propertyValue]*) INSERT INTO tbName queryStatement;
  • 13. Spark Streaming SQL MERGE INTO mergeInto : MERGE INTO target=tableIdentifier tableAlias USING (source=tableIdentifier (timeTravel)? | '(' subquery = query ')') tableAlias mergeCondition? matchedClauses* notMatchedClause? MERGE INTO target_table t USING source_table s ON s.id = t.id WHEN MATCHED AND s.opType = 'delete' THEN DELETE WHEN MATCHED AND s.opTye = 'update' THEN UPDATE SET id = s.id, name = s.name WHEN NOT MATCHED AND s.opType = 'insert' THEN INSERT (key, value) VALUES (key, value)
  • 14. Spark Streaming SQL DELAY / TUMBLING / HOPPING WHERE delay(colName) < 'duration'withWatermark("colName", "duration") SELECT avg(inv_quantity_on_hand) qoh FROM kafka_inventory WHERE delay(inv_data_time) < '2 minutes' GROUP BY TUMBLING (inv_data_time, interval 1 minute)
  • 15. Delta Lake Streaming/Batch Streaming/Batch ACID Transactions Metadata Management Unified Batch&Streaming Schema Enforcement&Evolution Update&Delete&Merge Time Travel Parquet Key Feature
  • 16. Delta Lake Improvement Delta Lake SparkSQL Spark Streaming SQL Update/Delete/ Optimize/Vacuum DDL/DML Hive/ Presto HDFS Alibaba OSS S3 …
  • 17. Delta Lake Improvement CREATE EXTERNAL TABLE delta_tbl(a string, b int) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'io.delta.hive.DeltaInputFormat' OUTPUTFORMAT 'io.delta.hive.DeltaOutputFormat' LOCATION 'oss://testbucket/delta/events'
  • 18. Spark Streaming SQL binlog ▪ no extra operational support for delta ▪ no load pressure on source database ▪ merge implement easily by SQL ▪ realtime low-latency(minute) CDC solution using Spark Streaming SQL & Delta Lake
  • 19. Spark Streaming SQL binlog CREATE SCAN cdctest_incremental_scan ON kafka_cdctest USING STREAM OPTIONS( startingOffsets='earliest', maxOffsetsPerTrigger='100000', failOnDataLoss=false ); CREATE STREAM cdctest_job OPTIONS(checkpointLocation='/delta/cdctest_checkpoint_oss') MERGE INTO delta_cdctest_oss as target USING ( SELECT // binlog parser … FROM cdctest_incremental_scan ) ON target.id = source.before_id WHEN MATCHED AND source.recordType='UPDATE' THEN UPDATE SET … WHEN MATCHED AND source.recordType='DELETE' THEN DELETE WHEN NOT MATCHED AND source.recordType='INSERT' THEN INSERT … CREATE TABLE kafka_cdctest USING KAFKA … CREATE TABLE delta_cdctest_oss USING DELTA … streaming-sql --master yarn --use-emr-datasource -f cdc_oss.sql 1 2 3 4
  • 20. Delta Table batch-batch- batch- …batch- DeltaTable .merge DeltaTable .merge DeltaTable .merge DeltaTable .merge Spark Streaming SQL MERGE INTO CDC solution using Spark Streaming SQL & Delta Lake
  • 21. Long Running Stability Improvement How to handle small files? ▪ increase batch interval(minutes) ▪ compcation(change data layout, not change data) ▪ adaptive execution mode
  • 22. Long Running Stability Improvement How to handle small files? - Scheduled Compaction Delta Table batch-batch- batch- … batch- DeltaTable .merge DeltaTable .merge DeltaTable .merge DeltaTable .merge CompactionCompaction scheduled job hourly/daily/… OPTIMIZE <tbl> [WHERE where_clause]
  • 23. Long Running Stability Improvement Problem: Streaming Job Failed when doing a compaction batch- Delta Table Compaction do transaction commit merge Transaction conflict check read How to handle small files? - Scheduled Compaction
  • 24. Long Running Stability Improvement Problem: Streaming Job Failed when doing a compaction binlog in batch streaming job status Improvement only insert Succeed Fix one bug: https://github.com/delta-io/delta/issues/326 including delete/update Failed streaming job retry this batch How to handle small files? - Scheduled Compaction
  • 25. Long Running Stability Improvement How to handle small files? - Auto Compaction Delta Table batch-batch- …batch- merge compaction compaction merge merge sequential execution, no conflict select files which file_size > COMPACT_FILE_SIZE flies number > TRIGGER_FILE_COU NT do compaction Yes No continue streaming Strategy
  • 26. Long Running Stability Improvement How to handle small files? - Adaptive Execution batch binlog target delta target changed files Join rewrite all changed files Join spark.sql.adaptive.enabled -> true Adaptive Excetion can auto merge small partitions, to decrease the number of reducers, then decrease the number of output files.
  • 27. Long Running Stability Improvement Performance issue batch-batch- batch- …batch- size of target delta table merge performance Runtime Filter https://issues.apache.org/jira/browse/SPARK-27227 batch binlog target delta target changed files Join filter
  • 29. Future ▪ auto schema change detected ▪ long running stable performance(read on merge) ▪ simplify user experience by SYNC grammar SYNC kafka_binlog_tbl TO delta_tbl OPTIONS( type='debezium.mysql' )
  • 30. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.