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

Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDatabricks
 
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
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in DeltaDatabricks
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta LakeDatabricks
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsAlluxio, Inc.
 
Building Lakehouses on Delta Lake with SQL Analytics Primer
Building Lakehouses on Delta Lake with SQL Analytics PrimerBuilding Lakehouses on Delta Lake with SQL Analytics Primer
Building Lakehouses on Delta Lake with SQL Analytics PrimerDatabricks
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsDatabricks
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLDatabricks
 
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
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroDatabricks
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsDatabricks
 
Delta Lake with Azure Databricks
Delta Lake with Azure DatabricksDelta Lake with Azure Databricks
Delta Lake with Azure DatabricksDustin Vannoy
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergFlink Forward
 
Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkDatabricks
 
[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
 
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
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
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
 

What's hot (20)

Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things Right
 
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
 
The delta architecture
The delta architectureThe delta architecture
The delta architecture
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Building Lakehouses on Delta Lake with SQL Analytics Primer
Building Lakehouses on Delta Lake with SQL Analytics PrimerBuilding Lakehouses on Delta Lake with SQL Analytics Primer
Building Lakehouses on Delta Lake with SQL Analytics Primer
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
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)
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
 
Delta Lake with Azure Databricks
Delta Lake with Azure DatabricksDelta Lake with Azure Databricks
Delta Lake with Azure Databricks
 
Spark architecture
Spark architectureSpark architecture
Spark architecture
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
 
Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache Spark
 
[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...
 
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...
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
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
 

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

Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...limedy534
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024thyngster
 
How we prevented account sharing with MFA
How we prevented account sharing with MFAHow we prevented account sharing with MFA
How we prevented account sharing with MFAAndrei Kaleshka
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesTimothy Spann
 
Vision, Mission, Goals and Objectives ppt..pptx
Vision, Mission, Goals and Objectives ppt..pptxVision, Mission, Goals and Objectives ppt..pptx
Vision, Mission, Goals and Objectives ppt..pptxellehsormae
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)jennyeacort
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 217djon017
 
Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Seán Kennedy
 
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...Biometric Authentication: The Evolution, Applications, Benefits and Challenge...
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...GQ Research
 
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhThiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhYasamin16
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our WorldEduminds Learning
 
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDINTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDRafezzaman
 
Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Colleen Farrelly
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Thomas Poetter
 
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档208367051
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Boston Institute of Analytics
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanMYRABACSAFRA2
 
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一F sss
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...Amil Baba Dawood bangali
 
Multiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfMultiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfchwongval
 

Recently uploaded (20)

Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
 
How we prevented account sharing with MFA
How we prevented account sharing with MFAHow we prevented account sharing with MFA
How we prevented account sharing with MFA
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
 
Vision, Mission, Goals and Objectives ppt..pptx
Vision, Mission, Goals and Objectives ppt..pptxVision, Mission, Goals and Objectives ppt..pptx
Vision, Mission, Goals and Objectives ppt..pptx
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2
 
Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...Student profile product demonstration on grades, ability, well-being and mind...
Student profile product demonstration on grades, ability, well-being and mind...
 
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...Biometric Authentication: The Evolution, Applications, Benefits and Challenge...
Biometric Authentication: The Evolution, Applications, Benefits and Challenge...
 
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhThiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our World
 
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTDINTERNSHIP ON PURBASHA COMPOSITE TEX LTD
INTERNSHIP ON PURBASHA COMPOSITE TEX LTD
 
Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024Generative AI for Social Good at Open Data Science East 2024
Generative AI for Social Good at Open Data Science East 2024
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
 
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population Mean
 
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
 
Multiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfMultiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdf
 

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.