SlideShare a Scribd company logo
1 of 75
Download to read offline
Designing and
Implementing Real-time
Data Lake with
Dynamically changing
Schemas
Agenda
Mate Gulyas
Practice Lead and Principal Instructor
Databricks
Shasidhar Eranti
Resident Solutions Engineer
Databricks
Introduction
▪ SEGA is a worldwide leader in interactive entertainment
▪ SEGA is a worldwide leader in interactive entertainment
▪ Huge franchises including Sonic, Total War and Football
Manager
▪ SEGA is a worldwide leader in interactive entertainment
▪ Huge franchises including Sonic, Total War and Football
Manager
▪ SEGA is currently celebrating its long awaited 60th anniversary.
▪ SEGA is a worldwide leader in interactive entertainment
▪ Huge franchises including Sonic, Total War and Football
Manager
▪ SEGA is currently celebrating its long awaited 60th anniversary.
▪ SEGA also produces arcade machines, holiday resorts, films
and merchandise
▪ Real time data from SEGA titles is crucial for business users.
▪ Real time data from SEGA titles is crucial for business users.
▪ SEGA’s 6 studios send data to one centralised data platform.
▪ Real time data from SEGA titles is crucial for business users.
▪ SEGA’s 6 studios send data to one centralised data platform.
▪ New events are frequently added and event schemas evolve
overtime.
▪ Real time data from SEGA titles is crucial for business users.
▪ SEGA’s 6 studios send data to one centralised data platform.
▪ New events are frequently added and event schemas evolve
overtime.
▪ Over 300 event types from over 40 SEGA titles (constantly growing)
▪ Real time data from SEGA titles is crucial for business users.
▪ SEGA’s 6 studios send data to one centralised data platform.
▪ New events are frequently added and event schemas evolve
overtime.
▪ Over 300 event types from over 40 SEGA titles (constantly growing)
▪ Events arrive at a rate of 8,000 every second
What is the GOAL and the CHALLENGE we try to
achieve?
Real time
data lake
No upfront
information about
the schemas or the
upcoming schema
changes
No downtime
Architecture
Key Requirements
Ingest different
types of JSON at
scale
Handle schema
evolution
dynamically
Serve
un-structured
data in a
structured form
for Business
users
Architecture
● Delta Architecture (Bronze - Silver
layers)
Architecture
● Delta Architecture (Bronze - Silver
layers)
Architecture
● Ingestion Stream (Bronze)
Using forEachBatch()
○ Dump JSON into delta table
○ Track Schema changes
● Delta Architecture (Bronze - Silver
layers)
Architecture
● Ingestion Stream (Bronze)
Using forEachBatch()
○ Dump JSON into delta table
○ Track Schema changes
● Stream multiplexing using Delta
● Event Streams(Silver)
○ Read from Bronze table
○ Fetch event schema
○ Apply schema using from_json()
○ Write to Silver table
● Delta Architecture (Bronze - Silver
layers)
Architecture
● Ingestion Stream (Bronze)
Using forEachBatch()
○ Dump JSON into delta table
○ Track Schema changes
● Stream multiplexing using Delta
● Event Streams(Silver)
○ Read from Bronze table
○ Fetch event schema
○ Apply schema using from_json()
○ Write to Silver table
Sample Data
Bronze Table
Sample Data
Silver Tables
Bronze Table
Event Type 1.1 Event Type 2.1
Schema Inference
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
}
Schema Changes
{
“event_type”: “1.1”,
“user_agent”: “firefox”,
“has_plugins”: “true”,
}
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
}
Schema Variation Hash
1. Raw message
{
“event_type”: “1.1”,
“user_agent”: “chrome”
}
Schema Variation Hash
[“event_type”, “user_agent”]
1. Raw message 2. Sorted list of ALL columns (including nested)
{
“event_type”: “1.1”,
“user_agent”: “chrome”
}
Schema Variation Hash
7862AF20813560D9AAEAF38D7E
[“event_type”, “user_agent”]
3. Calculate SHA1 Hash
1. Raw message 2. Sorted list of ALL columns (including nested)
Schema Repository
Schema Repository
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
“has_plugins”: “true”,
}
Schema Variation Hash
1. Raw message
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
“has_plugins”: “true”,
}
Schema Variation Hash
[“event_type”, “user_agent”, ”has_plugins”]
1. Raw message 2. Sorted list of ALL columns (including nested)
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
“has_plugins”: “true”,
}
Schema Variation Hash
BEA2ACAF2081350D9AAEAF38D7E
[“event_type”, “user_agent”, ”has_plugins”]
3. Calculate SHA1 Hash
1. Raw message 2. Sorted list of ALL columns (including nested)
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
“has_plugins”: “true”,
}
Schema Variation Hash
BEA2ACAF2081350D9AAEAF38D7E
[“event_type”, “user_agent”, ”has_plugins”]
3. Calculate SHA1 Hash
1. Raw message 2. Sorted list of ALL columns (including nested)
Not in Schema Repository
{
“event_type”: “1.1”,
“user_agent”: “chrome”,
“has_plugins”: “true”,
}
Schema Variation Hash
BEA2ACAF2081350D9AAEAF38D7E
[“event_type”, “user_agent”, ”has_plugins”]
3. Calculate SHA1 Hash
1. Raw message 2. Sorted list of ALL columns (including nested)
Not in Schema Repository
We need to update the schema
for 1.1
Foreach Batch
Update the Schema
Update the Schema
The new, so far UNSEEN message
Update the Schema
All of the the old prototypes from the Schema Repository
(We have only 1 now, but could be more)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
from typing import List
from pyspark.sql import Row
def inferSchema(protoPayloads: List[str]) -> "DataType":
schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads))
return spark
.read
.option("inferSchema", True)
.json(schemaProtoDF.rdd.map(lambda r: r.json))
.schema)
Update the Schema
Schema Repository
Schema Repository
We now have a new schema that incorporates all
the previous prototypes from all known schema
variations
Silver tables
Foreach Batch
def assert_and_process(event_type: String, target: String) (df:DataFrame,
batchId: Long): Unit = {
val (schema, schemaVersion) = get_schema(schema_repository, event_type_id)
df
.transform(process_raw_events(schema, schemaVersion))
.write.format("delta").mode("append")
.option("mergeSchema", true)
.save(target)
}
Retrieve the schema
def assert_and_process(event_type: String, target: String)(df:DataFrame,
batchId: Long): Unit = {
val (schema, schemaVersion) = get_schema(schema_repository, event_type_id)
df
.transform(process_raw_events(schema, schemaVersion))
.write.format("delta").mode("append")
.option("mergeSchema", true)
.save(target)
}
Retrieve the schema
def assert_and_process(event_type: String, target: String)(df:DataFrame,
batchId: Long): Unit = {
val (schema, schemaVersion) = get_schema(schema_repository, event_type_id)
df
.transform(process_raw_events(schema, schemaVersion))
.write.format("delta").mode("append")
.option("mergeSchema", true)
.save(target)
}
Retrieve the schema
def assert_and_process(event_type: String, target: String)(df:DataFrame,
batchId: Long): Unit = {
val (schema, schemaVersion) = get_schema(schema_repository, event_type_id)
df
.transform(process_raw_events(schema, schemaVersion))
.write.format("delta").mode("append").partitionBy(partitionColumns: _*)
.option("mergeSchema", true)
.save(target)
}
Retrieve the schema
Productionization
(Deployment and Monitoring)
Deploying Event Streams
Deploying Event Streams
● Events are grouped logically
Deploying Event Streams
● Events are grouped logically
● Stream groups are deployed on job
clusters
Deploying Event Streams
● Events are grouped logically
● Stream groups are deployed on job
clusters
● Two main aspects
○ Schema change
○ New Schema detected
Deploying Event Streams
● Events are grouped logically
● Stream groups are deployed on job
clusters
● Two main aspects
○ Schema change
○ New Schema detected
Schema change
● Incompatible schema changes causes
stream failures
Deploying Event Streams
● Events are grouped logically
● Stream groups are deployed on job
clusters
● Two main aspects
○ Schema change
○ New Schema detected
Schema change
● Incompatible schema changes causes
stream failures
● Stream monitoring in job clusters
Deploying Event Streams
● Events are grouped logically
● Stream groups are deployed on job
clusters
● Two main aspects
○ Schema change
○ New Schema detected
Schema change
● Incompatible schema changes causes
stream failures
● Stream monitoring in job clusters
New Schema detected
Management Stream EventGroup table
Management Stream EventGroup table
● Tracks schema changes from
schemaRegistry table
Management Stream EventGroup table
● Tracks schema changes from
schemaRegistry table
● Two type of source changes
○ Change in schema
○ New schema detected
Management Stream EventGroup table
● Tracks schema changes from
schemaRegistry table
● Two type of source changes
○ Change in schema
○ New schema detected
● Change in schema (No action)
Management Stream EventGroup table
● Tracks schema changes from
schemaRegistry table
● Two type of source changes
○ Change in schema
○ New schema detected
● Change in schema (No action)
● New schema detected
○ Add new entry in event group table
○ New stream is launched
automatically
Monitoring
Monitoring
● Use Structured Streaming listener
APIs to track metrics
Monitoring
● Use Structured Streaming listener
APIs to track metrics
● Dump Streaming metrics to central
dashboarding tool
Monitoring
● Use Structured Streaming listener
APIs to track metrics
● Dump Streaming metrics to central
dashboarding tool
● Key metrics tracked in monitoring
dashboard
○ Stream Status
○ Streaming latency
Monitoring
● Use Structured Streaming listener
APIs to track metrics
● Dump Streaming metrics to central
dashboarding tool
● Key metrics tracked in monitoring
dashboard
○ Stream Status
○ Streaming latency
● Enable Stream metrics capture for
ganglia using
spark.sql.streaming.metricsEnabled=true
Key takeaways
Delta helps with
Schema Evolution
and Stream
Multiplexing
capabilities
Schema Variation
hash to detect
schema changes
ImplementationArchitecture
Job clusters to run
streams in
production
Productionizing
Felix Baker, SEGA
”
“This has revolutionised the flow of analytics from our games
and has enabled business users to analyse and react to data
far more quickly than we have been able to do previously.”
Feedback
Your feedback is important to us.
Don’t forget to rate
and review the sessions.

More Related Content

What's hot

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.
 
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...Databricks
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeDatabricks
 
The columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowThe columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowJulien Le Dem
 
Introduction to Apache Kudu
Introduction to Apache KuduIntroduction to Apache Kudu
Introduction to Apache KuduJeff Holoman
 
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
 
Better Together: How Graph database enables easy data integration with Spark ...
Better Together: How Graph database enables easy data integration with Spark ...Better Together: How Graph database enables easy data integration with Spark ...
Better Together: How Graph database enables easy data integration with Spark ...TigerGraph
 
Scaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksScaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksDatabricks
 
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
 
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...Spark Summit
 
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
 
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...HostedbyConfluent
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsGuido Schmutz
 
Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3DataWorks Summit
 
Databricks Fundamentals
Databricks FundamentalsDatabricks Fundamentals
Databricks FundamentalsDalibor Wijas
 
Parallelization of Structured Streaming Jobs Using Delta Lake
Parallelization of Structured Streaming Jobs Using Delta LakeParallelization of Structured Streaming Jobs Using Delta Lake
Parallelization of Structured Streaming Jobs Using Delta LakeDatabricks
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkFlink Forward
 
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...Flink Forward
 

What's hot (20)

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
 
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
The columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowThe columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache Arrow
 
Rds data lake @ Robinhood
Rds data lake @ Robinhood Rds data lake @ Robinhood
Rds data lake @ Robinhood
 
Introduction to Apache Kudu
Introduction to Apache KuduIntroduction to Apache Kudu
Introduction to Apache Kudu
 
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...
 
The delta architecture
The delta architectureThe delta architecture
The delta architecture
 
Better Together: How Graph database enables easy data integration with Spark ...
Better Together: How Graph database enables easy data integration with Spark ...Better Together: How Graph database enables easy data integration with Spark ...
Better Together: How Graph database enables easy data integration with Spark ...
 
Scaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksScaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on Databricks
 
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
 
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
 
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
 
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka Streams
 
Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3
 
Databricks Fundamentals
Databricks FundamentalsDatabricks Fundamentals
Databricks Fundamentals
 
Parallelization of Structured Streaming Jobs Using Delta Lake
Parallelization of Structured Streaming Jobs Using Delta LakeParallelization of Structured Streaming Jobs Using Delta Lake
Parallelization of Structured Streaming Jobs Using Delta Lake
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
 

Similar to Designing and Implementing a Real-time Data Lake with Dynamically Changing Schema

WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0
WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0
WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0WSO2
 
Streaming Operational Data with MariaDB MaxScale
Streaming Operational Data with MariaDB MaxScaleStreaming Operational Data with MariaDB MaxScale
Streaming Operational Data with MariaDB MaxScaleMariaDB plc
 
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2
 
Timeseries - data visualization in Grafana
Timeseries - data visualization in GrafanaTimeseries - data visualization in Grafana
Timeseries - data visualization in GrafanaOCoderFest
 
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2
 
TSAR (TimeSeries AggregatoR) Tech Talk
TSAR (TimeSeries AggregatoR) Tech TalkTSAR (TimeSeries AggregatoR) Tech Talk
TSAR (TimeSeries AggregatoR) Tech TalkAnirudh Todi
 
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital EnterpriseWSO2
 
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor WSO2
 
Apache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextApache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextPrateek Maheshwari
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0HBaseCon
 
Deep learning and streaming in Apache Spark 2.2 by Matei Zaharia
Deep learning and streaming in Apache Spark 2.2 by Matei ZahariaDeep learning and streaming in Apache Spark 2.2 by Matei Zaharia
Deep learning and streaming in Apache Spark 2.2 by Matei ZahariaGoDataDriven
 
WSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsWSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsSriskandarajah Suhothayan
 
Intravert Server side processing for Cassandra
Intravert Server side processing for CassandraIntravert Server side processing for Cassandra
Intravert Server side processing for CassandraEdward Capriolo
 
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"DataStax Academy
 
Grid Objects in InduSoft Web Studio
Grid Objects in InduSoft Web StudioGrid Objects in InduSoft Web Studio
Grid Objects in InduSoft Web StudioAVEVA
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0Petr Zapletal
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Flink Forward
 

Similar to Designing and Implementing a Real-time Data Lake with Dynamically Changing Schema (20)

WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0
WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0
WSO2 Product Release Webinar: WSO2 Complex Event Processor 4.0
 
Streaming Operational Data with MariaDB MaxScale
Streaming Operational Data with MariaDB MaxScaleStreaming Operational Data with MariaDB MaxScale
Streaming Operational Data with MariaDB MaxScale
 
Clojure@Nuday
Clojure@NudayClojure@Nuday
Clojure@Nuday
 
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
 
Timeseries - data visualization in Grafana
Timeseries - data visualization in GrafanaTimeseries - data visualization in Grafana
Timeseries - data visualization in Grafana
 
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
 
TSAR (TimeSeries AggregatoR) Tech Talk
TSAR (TimeSeries AggregatoR) Tech TalkTSAR (TimeSeries AggregatoR) Tech Talk
TSAR (TimeSeries AggregatoR) Tech Talk
 
Tsar tech talk
Tsar tech talkTsar tech talk
Tsar tech talk
 
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise
[WSO2Con EU 2017] Streaming Analytics Patterns for Your Digital Enterprise
 
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor
WSO2 Product Release Webinar - Introducing the WSO2 Complex Event Processor
 
Apache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextApache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's Next
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0
 
Deep learning and streaming in Apache Spark 2.2 by Matei Zaharia
Deep learning and streaming in Apache Spark 2.2 by Matei ZahariaDeep learning and streaming in Apache Spark 2.2 by Matei Zaharia
Deep learning and streaming in Apache Spark 2.2 by Matei Zaharia
 
WSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsWSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needs
 
Intravert Server side processing for Cassandra
Intravert Server side processing for CassandraIntravert Server side processing for Cassandra
Intravert Server side processing for Cassandra
 
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"
NYC* 2013 - "Advanced Data Processing: Beyond Queries and Slices"
 
Building Streaming Applications with Streaming SQL
Building Streaming Applications with Streaming SQLBuilding Streaming Applications with Streaming SQL
Building Streaming Applications with Streaming SQL
 
Grid Objects in InduSoft Web Studio
Grid Objects in InduSoft Web StudioGrid Objects in InduSoft Web Studio
Grid Objects in InduSoft Web Studio
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
 

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

DATA ANALYSIS using various data sets like shoping data set etc
DATA ANALYSIS using various data sets like shoping data set etcDATA ANALYSIS using various data sets like shoping data set etc
DATA ANALYSIS using various data sets like shoping data set etclalithasri22
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksdeepakthakur548787
 
Statistics For Management by Richard I. Levin 8ed.pdf
Statistics For Management by Richard I. Levin 8ed.pdfStatistics For Management by Richard I. Levin 8ed.pdf
Statistics For Management by Richard I. Levin 8ed.pdfnikeshsingh56
 
Non Text Magic Studio Magic Design for Presentations L&P.pdf
Non Text Magic Studio Magic Design for Presentations L&P.pdfNon Text Magic Studio Magic Design for Presentations L&P.pdf
Non Text Magic Studio Magic Design for Presentations L&P.pdfPratikPatil591646
 
Role of Consumer Insights in business transformation
Role of Consumer Insights in business transformationRole of Consumer Insights in business transformation
Role of Consumer Insights in business transformationAnnie Melnic
 
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelDecoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelBoston Institute of Analytics
 
Presentation of project of business person who are success
Presentation of project of business person who are successPresentation of project of business person who are success
Presentation of project of business person who are successPratikSingh115843
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBoston Institute of Analytics
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfblazblazml
 
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...Jack Cole
 
Digital Indonesia Report 2024 by We Are Social .pdf
Digital Indonesia Report 2024 by We Are Social .pdfDigital Indonesia Report 2024 by We Are Social .pdf
Digital Indonesia Report 2024 by We Are Social .pdfNicoChristianSunaryo
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Boston Institute of Analytics
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...Dr Arash Najmaei ( Phd., MBA, BSc)
 
IBEF report on the Insurance market in India
IBEF report on the Insurance market in IndiaIBEF report on the Insurance market in India
IBEF report on the Insurance market in IndiaManalVerma4
 

Recently uploaded (17)

DATA ANALYSIS using various data sets like shoping data set etc
DATA ANALYSIS using various data sets like shoping data set etcDATA ANALYSIS using various data sets like shoping data set etc
DATA ANALYSIS using various data sets like shoping data set etc
 
Data Analysis Project: Stroke Prediction
Data Analysis Project: Stroke PredictionData Analysis Project: Stroke Prediction
Data Analysis Project: Stroke Prediction
 
Insurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis ProjectInsurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis Project
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing works
 
Statistics For Management by Richard I. Levin 8ed.pdf
Statistics For Management by Richard I. Levin 8ed.pdfStatistics For Management by Richard I. Levin 8ed.pdf
Statistics For Management by Richard I. Levin 8ed.pdf
 
Non Text Magic Studio Magic Design for Presentations L&P.pdf
Non Text Magic Studio Magic Design for Presentations L&P.pdfNon Text Magic Studio Magic Design for Presentations L&P.pdf
Non Text Magic Studio Magic Design for Presentations L&P.pdf
 
Role of Consumer Insights in business transformation
Role of Consumer Insights in business transformationRole of Consumer Insights in business transformation
Role of Consumer Insights in business transformation
 
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelDecoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
 
Presentation of project of business person who are success
Presentation of project of business person who are successPresentation of project of business person who are success
Presentation of project of business person who are success
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
 
2023 Survey Shows Dip in High School E-Cigarette Use
2023 Survey Shows Dip in High School E-Cigarette Use2023 Survey Shows Dip in High School E-Cigarette Use
2023 Survey Shows Dip in High School E-Cigarette Use
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
 
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
 
Digital Indonesia Report 2024 by We Are Social .pdf
Digital Indonesia Report 2024 by We Are Social .pdfDigital Indonesia Report 2024 by We Are Social .pdf
Digital Indonesia Report 2024 by We Are Social .pdf
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
 
IBEF report on the Insurance market in India
IBEF report on the Insurance market in IndiaIBEF report on the Insurance market in India
IBEF report on the Insurance market in India
 

Designing and Implementing a Real-time Data Lake with Dynamically Changing Schema

  • 1. Designing and Implementing Real-time Data Lake with Dynamically changing Schemas
  • 2. Agenda Mate Gulyas Practice Lead and Principal Instructor Databricks Shasidhar Eranti Resident Solutions Engineer Databricks
  • 4. ▪ SEGA is a worldwide leader in interactive entertainment
  • 5. ▪ SEGA is a worldwide leader in interactive entertainment ▪ Huge franchises including Sonic, Total War and Football Manager
  • 6. ▪ SEGA is a worldwide leader in interactive entertainment ▪ Huge franchises including Sonic, Total War and Football Manager ▪ SEGA is currently celebrating its long awaited 60th anniversary.
  • 7. ▪ SEGA is a worldwide leader in interactive entertainment ▪ Huge franchises including Sonic, Total War and Football Manager ▪ SEGA is currently celebrating its long awaited 60th anniversary. ▪ SEGA also produces arcade machines, holiday resorts, films and merchandise
  • 8. ▪ Real time data from SEGA titles is crucial for business users.
  • 9. ▪ Real time data from SEGA titles is crucial for business users. ▪ SEGA’s 6 studios send data to one centralised data platform.
  • 10. ▪ Real time data from SEGA titles is crucial for business users. ▪ SEGA’s 6 studios send data to one centralised data platform. ▪ New events are frequently added and event schemas evolve overtime.
  • 11. ▪ Real time data from SEGA titles is crucial for business users. ▪ SEGA’s 6 studios send data to one centralised data platform. ▪ New events are frequently added and event schemas evolve overtime. ▪ Over 300 event types from over 40 SEGA titles (constantly growing)
  • 12. ▪ Real time data from SEGA titles is crucial for business users. ▪ SEGA’s 6 studios send data to one centralised data platform. ▪ New events are frequently added and event schemas evolve overtime. ▪ Over 300 event types from over 40 SEGA titles (constantly growing) ▪ Events arrive at a rate of 8,000 every second
  • 13. What is the GOAL and the CHALLENGE we try to achieve? Real time data lake No upfront information about the schemas or the upcoming schema changes No downtime
  • 15. Key Requirements Ingest different types of JSON at scale Handle schema evolution dynamically Serve un-structured data in a structured form for Business users
  • 17. ● Delta Architecture (Bronze - Silver layers) Architecture
  • 18. ● Delta Architecture (Bronze - Silver layers) Architecture ● Ingestion Stream (Bronze) Using forEachBatch() ○ Dump JSON into delta table ○ Track Schema changes
  • 19. ● Delta Architecture (Bronze - Silver layers) Architecture ● Ingestion Stream (Bronze) Using forEachBatch() ○ Dump JSON into delta table ○ Track Schema changes ● Stream multiplexing using Delta ● Event Streams(Silver) ○ Read from Bronze table ○ Fetch event schema ○ Apply schema using from_json() ○ Write to Silver table
  • 20. ● Delta Architecture (Bronze - Silver layers) Architecture ● Ingestion Stream (Bronze) Using forEachBatch() ○ Dump JSON into delta table ○ Track Schema changes ● Stream multiplexing using Delta ● Event Streams(Silver) ○ Read from Bronze table ○ Fetch event schema ○ Apply schema using from_json() ○ Write to Silver table
  • 22. Sample Data Silver Tables Bronze Table Event Type 1.1 Event Type 2.1
  • 24. { “event_type”: “1.1”, “user_agent”: “chrome”, } Schema Changes { “event_type”: “1.1”, “user_agent”: “firefox”, “has_plugins”: “true”, }
  • 26. { “event_type”: “1.1”, “user_agent”: “chrome” } Schema Variation Hash [“event_type”, “user_agent”] 1. Raw message 2. Sorted list of ALL columns (including nested)
  • 27. { “event_type”: “1.1”, “user_agent”: “chrome” } Schema Variation Hash 7862AF20813560D9AAEAF38D7E [“event_type”, “user_agent”] 3. Calculate SHA1 Hash 1. Raw message 2. Sorted list of ALL columns (including nested)
  • 30. { “event_type”: “1.1”, “user_agent”: “chrome”, “has_plugins”: “true”, } Schema Variation Hash 1. Raw message
  • 31. { “event_type”: “1.1”, “user_agent”: “chrome”, “has_plugins”: “true”, } Schema Variation Hash [“event_type”, “user_agent”, ”has_plugins”] 1. Raw message 2. Sorted list of ALL columns (including nested)
  • 32. { “event_type”: “1.1”, “user_agent”: “chrome”, “has_plugins”: “true”, } Schema Variation Hash BEA2ACAF2081350D9AAEAF38D7E [“event_type”, “user_agent”, ”has_plugins”] 3. Calculate SHA1 Hash 1. Raw message 2. Sorted list of ALL columns (including nested)
  • 33. { “event_type”: “1.1”, “user_agent”: “chrome”, “has_plugins”: “true”, } Schema Variation Hash BEA2ACAF2081350D9AAEAF38D7E [“event_type”, “user_agent”, ”has_plugins”] 3. Calculate SHA1 Hash 1. Raw message 2. Sorted list of ALL columns (including nested) Not in Schema Repository
  • 34. { “event_type”: “1.1”, “user_agent”: “chrome”, “has_plugins”: “true”, } Schema Variation Hash BEA2ACAF2081350D9AAEAF38D7E [“event_type”, “user_agent”, ”has_plugins”] 3. Calculate SHA1 Hash 1. Raw message 2. Sorted list of ALL columns (including nested) Not in Schema Repository We need to update the schema for 1.1
  • 37. Update the Schema The new, so far UNSEEN message
  • 38. Update the Schema All of the the old prototypes from the Schema Repository (We have only 1 now, but could be more)
  • 40. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 41. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 42. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 43. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 44. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 45. from typing import List from pyspark.sql import Row def inferSchema(protoPayloads: List[str]) -> "DataType": schemaProtoDF = spark.createDataFrame(map(lambda x: Row(json=x), protoPayloads)) return spark .read .option("inferSchema", True) .json(schemaProtoDF.rdd.map(lambda r: r.json)) .schema) Update the Schema
  • 48. We now have a new schema that incorporates all the previous prototypes from all known schema variations
  • 51. def assert_and_process(event_type: String, target: String) (df:DataFrame, batchId: Long): Unit = { val (schema, schemaVersion) = get_schema(schema_repository, event_type_id) df .transform(process_raw_events(schema, schemaVersion)) .write.format("delta").mode("append") .option("mergeSchema", true) .save(target) } Retrieve the schema
  • 52. def assert_and_process(event_type: String, target: String)(df:DataFrame, batchId: Long): Unit = { val (schema, schemaVersion) = get_schema(schema_repository, event_type_id) df .transform(process_raw_events(schema, schemaVersion)) .write.format("delta").mode("append") .option("mergeSchema", true) .save(target) } Retrieve the schema
  • 53. def assert_and_process(event_type: String, target: String)(df:DataFrame, batchId: Long): Unit = { val (schema, schemaVersion) = get_schema(schema_repository, event_type_id) df .transform(process_raw_events(schema, schemaVersion)) .write.format("delta").mode("append") .option("mergeSchema", true) .save(target) } Retrieve the schema
  • 54. def assert_and_process(event_type: String, target: String)(df:DataFrame, batchId: Long): Unit = { val (schema, schemaVersion) = get_schema(schema_repository, event_type_id) df .transform(process_raw_events(schema, schemaVersion)) .write.format("delta").mode("append").partitionBy(partitionColumns: _*) .option("mergeSchema", true) .save(target) } Retrieve the schema
  • 57. Deploying Event Streams ● Events are grouped logically
  • 58. Deploying Event Streams ● Events are grouped logically ● Stream groups are deployed on job clusters
  • 59. Deploying Event Streams ● Events are grouped logically ● Stream groups are deployed on job clusters ● Two main aspects ○ Schema change ○ New Schema detected
  • 60. Deploying Event Streams ● Events are grouped logically ● Stream groups are deployed on job clusters ● Two main aspects ○ Schema change ○ New Schema detected Schema change ● Incompatible schema changes causes stream failures
  • 61. Deploying Event Streams ● Events are grouped logically ● Stream groups are deployed on job clusters ● Two main aspects ○ Schema change ○ New Schema detected Schema change ● Incompatible schema changes causes stream failures ● Stream monitoring in job clusters
  • 62. Deploying Event Streams ● Events are grouped logically ● Stream groups are deployed on job clusters ● Two main aspects ○ Schema change ○ New Schema detected Schema change ● Incompatible schema changes causes stream failures ● Stream monitoring in job clusters New Schema detected
  • 64. Management Stream EventGroup table ● Tracks schema changes from schemaRegistry table
  • 65. Management Stream EventGroup table ● Tracks schema changes from schemaRegistry table ● Two type of source changes ○ Change in schema ○ New schema detected
  • 66. Management Stream EventGroup table ● Tracks schema changes from schemaRegistry table ● Two type of source changes ○ Change in schema ○ New schema detected ● Change in schema (No action)
  • 67. Management Stream EventGroup table ● Tracks schema changes from schemaRegistry table ● Two type of source changes ○ Change in schema ○ New schema detected ● Change in schema (No action) ● New schema detected ○ Add new entry in event group table ○ New stream is launched automatically
  • 69. Monitoring ● Use Structured Streaming listener APIs to track metrics
  • 70. Monitoring ● Use Structured Streaming listener APIs to track metrics ● Dump Streaming metrics to central dashboarding tool
  • 71. Monitoring ● Use Structured Streaming listener APIs to track metrics ● Dump Streaming metrics to central dashboarding tool ● Key metrics tracked in monitoring dashboard ○ Stream Status ○ Streaming latency
  • 72. Monitoring ● Use Structured Streaming listener APIs to track metrics ● Dump Streaming metrics to central dashboarding tool ● Key metrics tracked in monitoring dashboard ○ Stream Status ○ Streaming latency ● Enable Stream metrics capture for ganglia using spark.sql.streaming.metricsEnabled=true
  • 73. Key takeaways Delta helps with Schema Evolution and Stream Multiplexing capabilities Schema Variation hash to detect schema changes ImplementationArchitecture Job clusters to run streams in production Productionizing
  • 74. Felix Baker, SEGA ” “This has revolutionised the flow of analytics from our games and has enabled business users to analyse and react to data far more quickly than we have been able to do previously.”
  • 75. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.