SlideShare une entreprise Scribd logo
1  sur  90
Télécharger pour lire hors ligne
NoLambda: Combining Streaming,
Ad-Hoc, Machine Learning, and
Batch Analytics
 
andEvan Chan Helena Edelson
March 2016
Evan Chan
Distinguished Engineer,
User and contributor to Spark since 0.9, Cassandra since
0.6
Co-creator and maintainer of
Tuplejump
@evanfchan
http://velvia.github.io
Spark Job Server
Helena Edelson
|@helenaedelson github.com/helena
VP of Product Engineering,
Cloud Engineer, Big Data, Event-Driven systems
Committer: Kafka Connect Cassandra, Spark Cassandra
Connector
Contributor: Akka, Spring Integration
Speaker: Kafka Summit, Spark Summit, Strata, QCon, Scala
Days, Scala World, Philly ETE
Tuplejump
Tuplejump
is a big data technology leader providing solutions and
development partnership.
Tuplejump
Open Source: on GitHubTuplejump
- Distributed Spark + Cassandra analytics database
- Kafka-Cassandra Source and
Sink
- The rst Spark Cassandra integration
- Lucene indexer for Cassandra
- HDFS for Cassandra
FiloDB
Kafka Connect Cassandra
Calliope
Stargate
SnackFS
Tuplejump Consulting & Development
Tuplejump Data Blender
Topics
Modern streaming and batch/ad-hoc architectures
Pitfalls and Simpli cation
Precise and scalable streaming ingestion
FiloDB and fast analytics with competitive storage
cost
Machine learning with Spark, Cassandra, and FiloDB
Delivering Meaning
Derived From
Many data streams
Disparate sources and schemas
Originating from many
locations
The Problem Domain
Build scalable, adaptable, self-healing, distributed data
processing systems for
24 / 7 Uptime
Auto scale out
Complex analytics and learning tasks
Aggregate global data
Ops for global, multi-dc clustered data
ows
Decoupled services
Idempotent & Acceptable Consistency
No data loss
Factors & Constraints in Architecture
Corner cases, Industry use cases
Legal constraints - user data crossing
borders
% Writes on ingestion, % Reads
How much in memory?
Counters? Geo-locational?
JDK version for cloud deployments
Dimensions of data in queries
What needs real time feedback loops
The list goes on...
Need Self-Healing Systems
Massive event spikes & bursty traf c
Fast producers / slow consumers
Network partitioning & out of sync
systems
DC down
Not DDOS'ing ourselves from fast streams
No data loss when auto-scaling down
Monitor Everything
Everything fails, all the time
Use Case
I need fast access to historical data on the y for predictive
modeling with real time data from the stream
Only, It's Not A Stream It's A Flood
Trillions of event writes per day
Billions of event reads per day
Massive events per second at
peak
Petabytes of total streaming data
Not All Streams Are Created The Same
Daily, Hourly, Frequency, Event Spikes at Peak, Overall Volume
Sub-second, low latency stream
processing
Higher latency stream processing
Scheduled or on request batch processing
Real Time
Just means Event Driven or processing events as they arrive
Doesn't automatically equal sub-second latency
requirements
Event Time
When an event is created, e.g. on sensor
Events should be uniquely timestamped on ingestion for
tracking, metrics and replay
Based on the schema of data in a given stream
Some can aggregate with sliding windows (T1...Tn,Tn+1...) using
window length + slide interval:
stream.reduceByKeyAndWindow((a:Int,b:Int)=>(a+b),Seconds(30),Seconds(10))
Some must aggregate by buckets
/Event_Type/YYYY/MM/DD/HH/MM/...
CREATETABLEtimeseries.raw_data_fu(
sensor_idtext,yearint,monthint,dayint,hourint,...
PRIMARYKEY((sensor_id),year,month,day,hour)
)WITHCLUSTERINGORDERBY(yearDESC,monthDESC,dayDESC,hourDESC);
Stream Processing
Kafka - Foundation to streaming
architecture
Samza - Just streaming
Gearpump - Real-time big data streaming
Analytics Stream Processing
Storm - Real-time Analytics, ML, needs Trident to
stream
Flink - Real-time Analytics, ML, Graph
Spark Streaming - Micro-batch Analytics, ML, Graph
Legacy Infrastructure
Highly-invested-in existing architecture around
Hadoop
Existing analytics logic for scheduled MR jobs
Lambda Architecture
A data-processing architecture designed to handle massive quantities
of data by taking advantage of both batch and stream processing
methods.
Lambda Architecture
( )https://www.mapr.com/developercentral/lambda-architecture
λ The Good
Immutability - retaining master data
With timestamped events
Appended versus overwritten
events
Attempt to beat CAP
Pre-computed views for
further processing
faster ad-hoc querying
λ The Bad
Two Analytics systems to support
Operational complexity
By the time a scheduled job is run 90% of the data is stale
Many moving parts: KV store, real time platform, batch
technologies
Running similar code and reconciling queries in dual systems
Analytics logic changes on dual systems
λ The Overly Complicated
Immutable sequence of records is ingested and fed into
a batch processing system
and a stream processing
system
in parallel
Ultimately Very High TCO And...
Are Batch and Streaming Systems
Fundamentally Different?
Both accumulate events from *T1...Tn,
Tn+1...Tn+x,...*
Or bucketed by Year-Month-Day-Hour?
Streaming frameworks have schedulers
A Unified Streaming Architecture
Everything On The Streaming Platform
Scala / Spark
Streaming
Mesos
Akka
Cassandra
Kafka
SNACK (SMACK) Stack
High Throughput Distributed Messaging
High Scalability - billions of events per
day
Durability - no data loss
Immutability
Support Massive Number of Consumers
Very ef cient and low latency
Decouples Data Pipelines
Automatic recovery from broker failures
Stream Processing Simplified
Kafka Streams
In master, coming in v0.10
Removes the need to run another framework like Storm
alongside Kafka
Removes the need for separate infrastructures
Common stream operations, e.g. join, lter, map, etc.
Windowing
Proper time modeling, e.g. event time vs. processing time
Local state management with persistence and replication
Schema and Avro support
Spark Streaming
Iterative ML, Interactive Querying, Graph, DataFrames
One runtime for streaming and batch processing
Join streaming and static data sets
No code duplication
Easy Kafka stream integration
Easy to reconcile queries against multiple
sources
Easy integration of KV durable storage
Apache Cassandra
Horizontally scalable
Multi-Region / Multi-Datacenter
Always On - Survive regional outages
Extremely fast writes: - perfect for ingestion of real time /
machine data
Very exible data modelling (lists, sets, custom data types)
Easy to operate
Best of breed storage technology, huge community
BUT: Simple queries only
OLTP-oriented/center
High performance concurrency framework for Scala and
Java
Fault Tolerance
Asynchronous messaging and data processing
Parallelization
Location Transparency
Local / Remote Routing
Akka: Cluster / Persistence / Streams
Enables
Streaming and Batch In One System
Streaming ML and Analytics for Predictions In The Stream
Kafka Streams
valbuilder=newKStreamBuilder()
valstream:KStream[K,V]=builder.stream(des,des,"raw.data.topic")
.flatMapValues(value->Arrays.asList(value.toLowerCase.split("")
.map((k,v)->newKeyValue(k,v))
.countByKey(ser,ser,des,des,"kTable")
.toStream()
stream.to("results.topic",...)
valstreams=newKafkaStreams(builder,props)
streams.start()
 
https://github.com/con uentinc/demos
Spark Streaming Kafka
Immutable Raw Data From Kafka Stream
Replaying data streams: for fault tolerance, logic changes..
classKafkaStreamingActor(ssc:StreamingContext)extendsMyAggregationActor{

valstream=KafkaUtils.createDirectStream(...)
.map(RawWeatherData(_))
stream
.foreachRDD(_.toDF.write.format("filodb.spark")
.option("dataset","rawdata").save())
/*Pre-Aggregatedatainthestreamforfastqueryingandaggregationlater.*/
stream.map(hour=>
(hour.wsid,hour.year,hour.month,hour.day,hour.oneHourPrecip)
).saveToCassandra(timeseriesKeyspace,dailyPrecipTable)

}
Reading Data From Cassandra On Request, Further
Aggregation
Compute isolation in Akka Actor
classTemperatureActor(sc:SparkContext)extendsAggregationActor{

importakka.pattern.pipe
defreceive:Actor.Receive={

casee:GetMonthlyHiLowTemperature=>highLow(e,sender)

}


defhighLow(e:GetMonthlyHiLowTemperature,requester:ActorRef):Unit=

sc.cassandraTable[DailyTemperature](timeseriesKeyspace,dailyTempAggregTable)

.where("wsid=?ANDyear=?ANDmonth=?",e.wsid,e.year,e.month)
.collectAsync()

.map(MonthlyTemperature(_,e.wsid,e.year,e.month))pipeTorequester
}
Spark Streaming, MLLib
Kafka, Cassandra
valssc=newStreamingContext(sparkConf,Seconds(5)
)
valtestData=ssc.cassandraTable[String](keyspace,table)
.map(LabeledPoint.parse)
valtrainingStream=KafkaUtils.createDirectStream[_,_,_,_](..)
.map(transformFunc)
.map(LabeledPoint.parse)
trainingStream.saveToCassandra("ml_training_keyspace","raw_training_data")


valmodel=newStreamingLinearRegressionWithSGD()

.setInitialWeights(Vectors.dense(weights))

.trainOn(trainingStream)
model
.predictOnValues(testData.map(lp=>(lp.label,lp.features)))
.saveToCassandra("ml_predictions_keyspace","predictions")
What's Missing? One Pipeline For Fast +
Big Data
Using Cassandra for Batch Analytics /
Event Storage / ML?
Storage ef ciency and scan speeds for reading large volumes
of data (for complex analytics, ML) become important
concerns
Regular Cassandra CQL tables are not very good at either
storage ef ciency or scan speeds
A different, analytics-optimized solution is needed...
All hard work leads to pro t, but mere talk leads
to poverty.
- Proverbs 14:23
Introducing FiloDB
A distributed, versioned, columnar analytics database.
Built for Streaming.
 
github.com/tuplejump/FiloDB
Fast Analytics Storage
Scan speeds competitive with Apache Parquet
Up to 200x faster scan speeds than with Cassandra 2.x
Flexible ltering along two dimensions
Much more ef cient and exible partition key ltering
Ef cient columnar storage, up to 40x more ef cient than
Cassandra 2.x
Comparing Storage Costs and Query Speeds
https://www.oreilly.com/ideas/apache-cassandra-for-analytics-
a-performance-and-storage-analysis
Robust Distributed Storage
Apache Cassandra as the rock-solid storage engine. Scale out
with no SPOF. Cross-datacenter replication. Proven storage and
database technology.
 
Cassandra-Like Data Model
Column A Column B
Partition
key 1
Segment
1
Segment
2
Segment
1
Segment
2
Partition
key 2
Segment
1
Segment
2
Segment
1
Segment
2
partition keys - distributes data around a cluster, and allows
for ne grained and exible ltering
segment keys - do range scans within a partition, e.g. by time
slice
primary key based ingestion and updates
Flexible Filtering
Unlike Cassandra, FiloDB offers very exible and ef cient
ltering on partition keys. Partial key matches, fast IN queries on
any part of the partition key.
No need to write multiple tables to work around answering different
queries.
Spark SQL Queries!
CREATETABLEgdeltUSINGfilodb.sparkOPTIONS(dataset"gdelt");
SELECTActor1Name,Actor2Name,AvgToneFROMgdeltORDERBYAvgToneDESCLIMIT15
INSERTINTOgdeltSELECT*FROMNewMonthData;
Read to and write from Spark Dataframes
Append/merge to FiloDB table from Spark
Streaming
Use Tableau or any other JDBC tool
What's in the name?
Rich sweet layers of distributed, versioned database goodness
SNACK (SMACK) stack for all your
Analytics
Regular Cassandra tables for highly concurrent, aggregate /
key-value lookups (dashboards)
FiloDB + C* + Spark for ef cient long term event storage
Ad hoc / SQL / BI
Data source for MLLib / building models
Data storage for classi ed / predicted / scored data
Being Productionized as we speak...
One enterprise with many TB of nancial and reporting data is
moving their data warehouse to FiloDB + Cassandra + Spark
Another startup uses FiloDB as event storage, feeds the events
into Spark MLlib, scores incoming data, then stores the results
back in FiloDB for low-latency use cases
From their CTO: “I see close to MemSQL / Vertica or even
better” “More cost effective than Redshift”
FiloDB Use Cases
Data Warehousing / BI
< 10 second SLA, nontrivial reports, some concurrency
need to store and query lots of data ef ciently
Time series
idempotent write API, simultaneous write and read
workloads
In-memory SQL web server
700 queries per second using in-memory column store
FiloDB vs HDFS/Parquet
FiloDB Parquet
Ingestion Idempotent primary-key
based; appends and
replaces; deletes coming
File-based
append
API only
Filtering Partition-key and segment-
key ltering
Mostly
le-based
Scan
speeds
Parquet-like Good for
OLAP
Storage
cost
Within 35% of Parquet  
FiloDB vs HDFS/Parquet
In practice, with good data modeling, FiloDB is a far better t for
low-latency / concurrent BI / reporting / dashboard applications.
FiloDB vs Druid
Different use cases:
Druid is optimized mostly for OLAP cube / slice and dice
analysis. Append only, keeps only aggregates, not a raw event
store.
FiloDB stores raw data - can be used to build ML models,
visualize and analyze raw time series data, do complex event
ow analysis - much more exible
FiloDB can update/replace data
FiloDB does not require data denormalization - can handle
traditional BI star schemas with slowly changing dimension
tables
Come check out the demo!
Visit FiloDB at the Developer Showcase (Expo hall) today!
Machine Learning with Spark, Cassandra,
and FiloDB
Building a static model of NYC Taxi Trips
Predict time to get to destination based on pickup point, time
of day, other vars
Need to read all data (full table scan)
Dynamic models are better than static
models
Everything changes!
Continuously re ne model based on recent streaming data +
historical data + existing model
valssc=newStreamingContext(sparkConf,Seconds(5)
)
valdataStream=KafkaUtils.createDirectStream[..](..)
.map(transformFunc)
.map(LabeledPoint.parse)
dataStream.foreachRDD(_.toDF.write.format("filodb.spark")
.option("dataset","training").save())
if(trainNow){

varmodel=newStreamingLinearRegressionWithSGD()

.setInitialWeights(Vectors.dense(weights))

.trainOn(dataStream.join(historicalEvents))
}
model.predictOnValues(dataStream.map(lp=>(lp.label,lp.features)))
.insertIntoFilo("predictions")
The FiloDB Advantage for ML
Able to update dynamic models based on massive data
ow/updates
Integrate historical and recent events to build models
More data -> better models!
Can store scored raw data / predictions back in FiloDB
for fast user queries
FiloDB - Roadmap
Your input is appreciated!
Productionization and automated stress testing
Kafka input API / connector (without needing Spark)
In-memory caching for signi cant query speedup
True columnar querying and execution, using late
materialization and vectorization techniques. GPU/SIMD.
Projections. Often-repeated queries can be sped up
signi cantly with projections.
Thanks For Attending!
@helenaedelson
@evanfchan
@tuplejump
EXTRA SLIDES
What are my storage needs?
Non-persistent / in-memory: concurrent
viewers
Short term: latest trends
Longer term: raw event and aggregate storage
ML Models, predictions, scored data
Spark RDDs
Immutable, cache in memory and/or on
disk
Spark Streaming: UpdateStateByKey
IndexedRDD - can update bits of data
Snapshotting for recovery
Using Cassandra for Short Term Storage
1020s 1010s 1000s
Bus A Speed, GPS
Bus B
Bus C
Primary key = (Bus UUID, timestamp)
Easy queries: location and speed of single bus for a range of
time
Can also query most recent location + speed of all buses
(slower)
Data Warehousing with FiloDB
Scenarios
BI Reporting, concurrency + seconds latency
Ad-hoc queries
Needing to do JOINs with fact tables + dimension
tables
Slowly changing dim tables / hard to denormalize
Need to work with legacy BI tools
Real-world DW Architecture Stack
Ef cient columnar storage + ltering = low latency BI
Modeling Fact Tables for FiloDB
Single partition queries are really fast and take up only one
thread
Given the following two partition key columns:
entity_number, year_month
WHERE entity_number = '0453' AND
year_month = '2014 December'
Exact match for partition key is pushed down as one
partition
Consider the partition key carefully
Cassandra often requires multiple tables
What about the queries that do not translate to one partition?
Cassandra has many restrictions on partition key ltering (as of
2.x).
Table 1: partition key = (entity_number, year_month)
Can push down: WHERE entity_number = NN AND
year_month IN ('2014 Jan', '2014 Feb')as
well as equals
Table 2: partition key = (year_month, entity_number)
Can push down: WHERE year_month = YYMM AND
entity_number IN (123, 456)as well as equals
IN clause must be the last column to be pushed down. Two tables
are needed just for ef cient IN queries on either entity_number
or year_month.
FiloDB Flexible Partition Filters = WIN
With ONE table, FiloDB offers FAST, arbitrary partition key
ltering. All of the below are pushed down:
WHERE year_month IN ('2014 Jan', '2014 Feb')
(all entities)
WHERE entity_number = 146(all year months)
Any combo of =, IN
Space savings: 27 *2 = 54x
Multi-Table JOINs with just Cassandra
Sub-second Multi-Table JOINs with FiloDB
Sub-second Multi-Table JOINs with FiloDB
Four tables, all of them single-partition queries
Two tables were switched from regular Cassandra tables to
FiloDB tables. 40-60 columns each, ~60k items in partition.
Scan times went down from 5-6 seconds to < 250ms
For more details, please see this .Planet Cassandra blog post
Scalable Time-Series / Event Storage with
FiloDB
Designed for Streaming
New rows appended via Spark Streaming or Kafka
Writes are idempotent - easy exactly once ingestion
Converted to columnar chunks on ingest and stored in
C*
FiloDB keeps your data sorted as it is being ingested
Spark Streaming -> FiloDB
valratingsStream=KafkaUtils.createDirectStream[String,String,StringDecoder,Strin
ratingsStream.foreachRDD{
(message:RDD[(String,String)],batchTime:Time)=>{
valdf=message.map(_._2.split(",")).map(rating=>Rating(rating(0).trim.toInt,r
toDF("fromuserid","touserid","rating")
//addthebatchtimetotheDataFrame
valdfWithBatchTime=df.withColumn("batch_time",org.apache.spark.sql.functions.l
//savetheDataFrametoFiloDB
dfWithBatchTime.write.format("filodb.spark")
.option("dataset","ratings")
.save()
}
}
One-line change to write to FiloDB vs Cassandra
Modeling example: NYC Taxi Dataset
The public contains telemetry (pickup, dropoff
locations, times) info on millions of taxi rides in NYC.
NYC Taxi Dataset
Medallion pre x 1/1 - 1/6 1/7 - 1/12
AA records records
AB records records
Partition key - :stringPrefix medallion 2- hash
multiple drivers trips into ~300 partitions
Segment key - :timeslice pickup_datetime 6d
Row key - hack_license, pickup_datetime
Allows for easy ltering by individual drivers, and slicing by time.
DEMO TIME
New York City Taxi Data Demo (Spark Notebook)
To follow along:
https://github.com/tuplejump/FiloDB/blob/master/doc/FiloDB_Taxi_G
Fast, Updatable In-Memory
Columnar Storage
Unlike RDDs and DataFrames, FiloDB can ingest new data, and
still be fast
Unlike RDDs, FiloDB can lter in multiple ways, no need for
entire table scan
FAIR scheduler + sub-second latencies => web speed queries
700 Queries Per Second in Apache Spark!
Even for datasets with 15 million rows!
Using FiloDB's InMemoryColumnStore, single host / MBP,
5GB RAM
SQL to DataFrame caching
For more details, see .this blog post
FiloDB - How?
Multiple ways to Accelerate Queries
Columnar projection - read fewer columns, saves I/O
Partition key ltering - read less data
Sort key / PK ltering - read from subset of keys
Possible because FiloDB keeps data sorted
Versioning - write to multiple versions, read from the one you
choose
Cassandra CQL vs Columnar Layout
Cassandra stores CQL tables row-major, each row spans multiple
cells:
PartitionKey 01: rst 01:last 01:age 02: rst 02:last 02:age
Sales Bob Jones 34 Susan O'Connor 40
Engineering Dilbert P ? Dogbert Dog 1
 
Columnar layouts are column-major:
PartitionKey rst last age
Sales Bob, Susan Jones,
O'Connor
34,
40
Engineering Dilbert,
Dogbert
P, Dog ?, 1
FiloDB Cassandra Schema
CREATETABLEfilodb.gdelt_chunks(
partitiontext,
versionint,
columnnametext,
segmentidblob,
chunkidint,
datablob,
PRIMARYKEY((partition,version),columnname,segmentid,chunkid)
)WITHCLUSTERINGORDERBY(columnnameASC,segmentidASC,chunkidASC)
FiloDB Architecture
ColumnStore API - currently Cassandra and InMemory, you can
implement other backends - ElasticSearch? etc.

Contenu connexe

Tendances

Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...
Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...
Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...Helena Edelson
 
Sa introduction to big data pipelining with cassandra &amp; spark west mins...
Sa introduction to big data pipelining with cassandra &amp; spark   west mins...Sa introduction to big data pipelining with cassandra &amp; spark   west mins...
Sa introduction to big data pipelining with cassandra &amp; spark west mins...Simon Ambridge
 
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...Helena Edelson
 
Kick-Start with SMACK Stack
Kick-Start with SMACK StackKick-Start with SMACK Stack
Kick-Start with SMACK StackKnoldus Inc.
 
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Lucidworks
 
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationUsing Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationPatrick Di Loreto
 
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaLambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaHelena Edelson
 
Using the SDACK Architecture to Build a Big Data Product
Using the SDACK Architecture to Build a Big Data ProductUsing the SDACK Architecture to Build a Big Data Product
Using the SDACK Architecture to Build a Big Data ProductEvans Ye
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Evan Chan
 
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...Brian O'Neill
 
Realtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingRealtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingSantosh Sahoo
 
Lambda architecture with Spark
Lambda architecture with SparkLambda architecture with Spark
Lambda architecture with SparkVincent GALOPIN
 
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...Spark Summit
 
Developing a Real-time Engine with Akka, Cassandra, and Spray
Developing a Real-time Engine with Akka, Cassandra, and SprayDeveloping a Real-time Engine with Akka, Cassandra, and Spray
Developing a Real-time Engine with Akka, Cassandra, and SprayJacob Park
 
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Summit
 
Lambda Architecture Using SQL
Lambda Architecture Using SQLLambda Architecture Using SQL
Lambda Architecture Using SQLSATOSHI TAGOMORI
 
Real time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsReal time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsBen Laird
 
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion Dubai
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion DubaiSMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion Dubai
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion DubaiCodemotion Dubai
 
Bellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingBellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingSantosh Sahoo
 

Tendances (20)

Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...
Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...
Leveraging Kafka for Big Data in Real Time Bidding, Analytics, ML & Campaign ...
 
Sa introduction to big data pipelining with cassandra &amp; spark west mins...
Sa introduction to big data pipelining with cassandra &amp; spark   west mins...Sa introduction to big data pipelining with cassandra &amp; spark   west mins...
Sa introduction to big data pipelining with cassandra &amp; spark west mins...
 
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
 
Kick-Start with SMACK Stack
Kick-Start with SMACK StackKick-Start with SMACK Stack
Kick-Start with SMACK Stack
 
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
 
Lambda architecture
Lambda architectureLambda architecture
Lambda architecture
 
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationUsing Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
 
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaLambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
 
Using the SDACK Architecture to Build a Big Data Product
Using the SDACK Architecture to Build a Big Data ProductUsing the SDACK Architecture to Build a Big Data Product
Using the SDACK Architecture to Build a Big Data Product
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015
 
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...
Re-envisioning the Lambda Architecture : Web Services & Real-time Analytics ...
 
Realtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingRealtime Reporting using Spark Streaming
Realtime Reporting using Spark Streaming
 
Lambda architecture with Spark
Lambda architecture with SparkLambda architecture with Spark
Lambda architecture with Spark
 
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
 
Developing a Real-time Engine with Akka, Cassandra, and Spray
Developing a Real-time Engine with Akka, Cassandra, and SprayDeveloping a Real-time Engine with Akka, Cassandra, and Spray
Developing a Real-time Engine with Akka, Cassandra, and Spray
 
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
 
Lambda Architecture Using SQL
Lambda Architecture Using SQLLambda Architecture Using SQL
Lambda Architecture Using SQL
 
Real time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsReal time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.js
 
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion Dubai
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion DubaiSMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion Dubai
SMACK Stack - Fast Data Done Right by Stefan Siprell at Codemotion Dubai
 
Bellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingBellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark Streaming
 

Similaire à NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis

Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...
Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...
Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...Helena Edelson
 
Introduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterIntroduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterPaolo Castagna
 
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...GeeksLab Odessa
 
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdf
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdfDustinVannoy_DataPipelines_AzureDataConf_Dec22.pdf
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdfDustin Vannoy
 
Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Anton Nazaruk
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsAsis Mohanty
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
All Day DevOps - FLiP Stack for Cloud Data Lakes
All Day DevOps - FLiP Stack for Cloud Data LakesAll Day DevOps - FLiP Stack for Cloud Data Lakes
All Day DevOps - FLiP Stack for Cloud Data LakesTimothy Spann
 
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, Qlik
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, QlikKeeping Analytics Data Fresh in a Streaming Architecture | John Neal, Qlik
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, QlikHostedbyConfluent
 
Streaming Data Ingest and Processing with Apache Kafka
Streaming Data Ingest and Processing with Apache KafkaStreaming Data Ingest and Processing with Apache Kafka
Streaming Data Ingest and Processing with Apache KafkaAttunity
 
Time Series Analysis… using an Event Streaming Platform
Time Series Analysis… using an Event Streaming PlatformTime Series Analysis… using an Event Streaming Platform
Time Series Analysis… using an Event Streaming Platformconfluent
 
Time Series Analysis Using an Event Streaming Platform
 Time Series Analysis Using an Event Streaming Platform Time Series Analysis Using an Event Streaming Platform
Time Series Analysis Using an Event Streaming PlatformDr. Mirko Kämpf
 
DBA Fundamentals Group: Continuous SQL with Kafka and Flink
DBA Fundamentals Group: Continuous SQL with Kafka and FlinkDBA Fundamentals Group: Continuous SQL with Kafka and Flink
DBA Fundamentals Group: Continuous SQL with Kafka and FlinkTimothy Spann
 
What's new in Confluent 3.2 and Apache Kafka 0.10.2
What's new in Confluent 3.2 and Apache Kafka 0.10.2 What's new in Confluent 3.2 and Apache Kafka 0.10.2
What's new in Confluent 3.2 and Apache Kafka 0.10.2 confluent
 
AWS Webcast - Amazon Kinesis and Apache Storm
AWS Webcast - Amazon Kinesis and Apache StormAWS Webcast - Amazon Kinesis and Apache Storm
AWS Webcast - Amazon Kinesis and Apache StormAmazon Web Services
 
Streaming ETL with Apache Kafka and KSQL
Streaming ETL with Apache Kafka and KSQLStreaming ETL with Apache Kafka and KSQL
Streaming ETL with Apache Kafka and KSQLNick Dearden
 
Event Broker (Kafka) in a Modern Data Architecture
Event Broker (Kafka) in a Modern Data ArchitectureEvent Broker (Kafka) in a Modern Data Architecture
Event Broker (Kafka) in a Modern Data ArchitectureGuido Schmutz
 
Connect K of SMACK:pykafka, kafka-python or?
Connect K of SMACK:pykafka, kafka-python or?Connect K of SMACK:pykafka, kafka-python or?
Connect K of SMACK:pykafka, kafka-python or?Micron Technology
 
Leveraging Mainframe Data for Modern Analytics
Leveraging Mainframe Data for Modern AnalyticsLeveraging Mainframe Data for Modern Analytics
Leveraging Mainframe Data for Modern Analyticsconfluent
 
Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataDataWorks Summit/Hadoop Summit
 

Similaire à NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis (20)

Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...
Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...
Fast and Simplified Streaming, Ad-Hoc and Batch Analytics with FiloDB and Spa...
 
Introduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterIntroduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matter
 
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
AI&BigData Lab 2016. Сарапин Виктор: Размер имеет значение: анализ по требова...
 
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdf
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdfDustinVannoy_DataPipelines_AzureDataConf_Dec22.pdf
DustinVannoy_DataPipelines_AzureDataConf_Dec22.pdf
 
Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture Patterns
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
All Day DevOps - FLiP Stack for Cloud Data Lakes
All Day DevOps - FLiP Stack for Cloud Data LakesAll Day DevOps - FLiP Stack for Cloud Data Lakes
All Day DevOps - FLiP Stack for Cloud Data Lakes
 
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, Qlik
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, QlikKeeping Analytics Data Fresh in a Streaming Architecture | John Neal, Qlik
Keeping Analytics Data Fresh in a Streaming Architecture | John Neal, Qlik
 
Streaming Data Ingest and Processing with Apache Kafka
Streaming Data Ingest and Processing with Apache KafkaStreaming Data Ingest and Processing with Apache Kafka
Streaming Data Ingest and Processing with Apache Kafka
 
Time Series Analysis… using an Event Streaming Platform
Time Series Analysis… using an Event Streaming PlatformTime Series Analysis… using an Event Streaming Platform
Time Series Analysis… using an Event Streaming Platform
 
Time Series Analysis Using an Event Streaming Platform
 Time Series Analysis Using an Event Streaming Platform Time Series Analysis Using an Event Streaming Platform
Time Series Analysis Using an Event Streaming Platform
 
DBA Fundamentals Group: Continuous SQL with Kafka and Flink
DBA Fundamentals Group: Continuous SQL with Kafka and FlinkDBA Fundamentals Group: Continuous SQL with Kafka and Flink
DBA Fundamentals Group: Continuous SQL with Kafka and Flink
 
What's new in Confluent 3.2 and Apache Kafka 0.10.2
What's new in Confluent 3.2 and Apache Kafka 0.10.2 What's new in Confluent 3.2 and Apache Kafka 0.10.2
What's new in Confluent 3.2 and Apache Kafka 0.10.2
 
AWS Webcast - Amazon Kinesis and Apache Storm
AWS Webcast - Amazon Kinesis and Apache StormAWS Webcast - Amazon Kinesis and Apache Storm
AWS Webcast - Amazon Kinesis and Apache Storm
 
Streaming ETL with Apache Kafka and KSQL
Streaming ETL with Apache Kafka and KSQLStreaming ETL with Apache Kafka and KSQL
Streaming ETL with Apache Kafka and KSQL
 
Event Broker (Kafka) in a Modern Data Architecture
Event Broker (Kafka) in a Modern Data ArchitectureEvent Broker (Kafka) in a Modern Data Architecture
Event Broker (Kafka) in a Modern Data Architecture
 
Connect K of SMACK:pykafka, kafka-python or?
Connect K of SMACK:pykafka, kafka-python or?Connect K of SMACK:pykafka, kafka-python or?
Connect K of SMACK:pykafka, kafka-python or?
 
Leveraging Mainframe Data for Modern Analytics
Leveraging Mainframe Data for Modern AnalyticsLeveraging Mainframe Data for Modern Analytics
Leveraging Mainframe Data for Modern Analytics
 
Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing data
 

Plus de Helena Edelson

Toward Predictability and Stability
Toward Predictability and StabilityToward Predictability and Stability
Toward Predictability and StabilityHelena Edelson
 
Disorder And Tolerance In Distributed Systems At Scale
Disorder And Tolerance In Distributed Systems At ScaleDisorder And Tolerance In Distributed Systems At Scale
Disorder And Tolerance In Distributed Systems At ScaleHelena Edelson
 
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...Helena Edelson
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleHelena Edelson
 
Streaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and AkkaStreaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and AkkaHelena Edelson
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Helena Edelson
 
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Helena Edelson
 

Plus de Helena Edelson (8)

Toward Predictability and Stability
Toward Predictability and StabilityToward Predictability and Stability
Toward Predictability and Stability
 
Patterns In The Chaos
Patterns In The ChaosPatterns In The Chaos
Patterns In The Chaos
 
Disorder And Tolerance In Distributed Systems At Scale
Disorder And Tolerance In Distributed Systems At ScaleDisorder And Tolerance In Distributed Systems At Scale
Disorder And Tolerance In Distributed Systems At Scale
 
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...
Building Reactive Distributed Systems For Streaming Big Data, Analytics & Mac...
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For Scale
 
Streaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and AkkaStreaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and Akka
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
 
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
 

Dernier

CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):comworks
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLScyllaDB
 
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationBeyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationSafe Software
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024Stephanie Beckett
 
Powerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time ClashPowerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time Clashcharlottematthew16
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Mark Simos
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Commit University
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brandgvaughan
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesZilliz
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfSeasiaInfotech2
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024The Digital Insurer
 
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr LapshynFwdays
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostZilliz
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piececharlottematthew16
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxNavinnSomaal
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfAlex Barbosa Coqueiro
 

Dernier (20)

CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQL
 
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationBeyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024
 
Powerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time ClashPowerpoint exploring the locations used in television show Time Clash
Powerpoint exploring the locations used in television show Time Clash
 
DMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special EditionDMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special Edition
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!
 
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptxE-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brand
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector Databases
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdf
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024
 
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
"Federated learning: out of reach no matter how close",Oleksandr Lapshyn
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piece
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptx
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdf
 

NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis

  • 1. NoLambda: Combining Streaming, Ad-Hoc, Machine Learning, and Batch Analytics   andEvan Chan Helena Edelson March 2016
  • 2. Evan Chan Distinguished Engineer, User and contributor to Spark since 0.9, Cassandra since 0.6 Co-creator and maintainer of Tuplejump @evanfchan http://velvia.github.io Spark Job Server
  • 3. Helena Edelson |@helenaedelson github.com/helena VP of Product Engineering, Cloud Engineer, Big Data, Event-Driven systems Committer: Kafka Connect Cassandra, Spark Cassandra Connector Contributor: Akka, Spring Integration Speaker: Kafka Summit, Spark Summit, Strata, QCon, Scala Days, Scala World, Philly ETE Tuplejump
  • 4. Tuplejump is a big data technology leader providing solutions and development partnership. Tuplejump
  • 5. Open Source: on GitHubTuplejump - Distributed Spark + Cassandra analytics database - Kafka-Cassandra Source and Sink - The rst Spark Cassandra integration - Lucene indexer for Cassandra - HDFS for Cassandra FiloDB Kafka Connect Cassandra Calliope Stargate SnackFS
  • 8. Topics Modern streaming and batch/ad-hoc architectures Pitfalls and Simpli cation Precise and scalable streaming ingestion FiloDB and fast analytics with competitive storage cost Machine learning with Spark, Cassandra, and FiloDB
  • 9. Delivering Meaning Derived From Many data streams Disparate sources and schemas Originating from many locations
  • 10. The Problem Domain Build scalable, adaptable, self-healing, distributed data processing systems for 24 / 7 Uptime Auto scale out Complex analytics and learning tasks Aggregate global data Ops for global, multi-dc clustered data ows Decoupled services Idempotent & Acceptable Consistency No data loss
  • 11. Factors & Constraints in Architecture Corner cases, Industry use cases Legal constraints - user data crossing borders % Writes on ingestion, % Reads How much in memory? Counters? Geo-locational? JDK version for cloud deployments Dimensions of data in queries What needs real time feedback loops The list goes on...
  • 12. Need Self-Healing Systems Massive event spikes & bursty traf c Fast producers / slow consumers Network partitioning & out of sync systems DC down Not DDOS'ing ourselves from fast streams No data loss when auto-scaling down Monitor Everything Everything fails, all the time
  • 13. Use Case I need fast access to historical data on the y for predictive modeling with real time data from the stream
  • 14. Only, It's Not A Stream It's A Flood Trillions of event writes per day Billions of event reads per day Massive events per second at peak Petabytes of total streaming data
  • 15. Not All Streams Are Created The Same Daily, Hourly, Frequency, Event Spikes at Peak, Overall Volume Sub-second, low latency stream processing Higher latency stream processing Scheduled or on request batch processing
  • 16. Real Time Just means Event Driven or processing events as they arrive Doesn't automatically equal sub-second latency requirements Event Time When an event is created, e.g. on sensor Events should be uniquely timestamped on ingestion for tracking, metrics and replay
  • 17. Based on the schema of data in a given stream Some can aggregate with sliding windows (T1...Tn,Tn+1...) using window length + slide interval: stream.reduceByKeyAndWindow((a:Int,b:Int)=>(a+b),Seconds(30),Seconds(10)) Some must aggregate by buckets /Event_Type/YYYY/MM/DD/HH/MM/... CREATETABLEtimeseries.raw_data_fu( sensor_idtext,yearint,monthint,dayint,hourint,... PRIMARYKEY((sensor_id),year,month,day,hour) )WITHCLUSTERINGORDERBY(yearDESC,monthDESC,dayDESC,hourDESC);
  • 18. Stream Processing Kafka - Foundation to streaming architecture Samza - Just streaming Gearpump - Real-time big data streaming Analytics Stream Processing Storm - Real-time Analytics, ML, needs Trident to stream Flink - Real-time Analytics, ML, Graph Spark Streaming - Micro-batch Analytics, ML, Graph
  • 19. Legacy Infrastructure Highly-invested-in existing architecture around Hadoop Existing analytics logic for scheduled MR jobs
  • 20. Lambda Architecture A data-processing architecture designed to handle massive quantities of data by taking advantage of both batch and stream processing methods.
  • 22. λ The Good Immutability - retaining master data With timestamped events Appended versus overwritten events Attempt to beat CAP Pre-computed views for further processing faster ad-hoc querying
  • 23. λ The Bad Two Analytics systems to support Operational complexity By the time a scheduled job is run 90% of the data is stale Many moving parts: KV store, real time platform, batch technologies Running similar code and reconciling queries in dual systems Analytics logic changes on dual systems
  • 24. λ The Overly Complicated Immutable sequence of records is ingested and fed into a batch processing system and a stream processing system in parallel Ultimately Very High TCO And...
  • 25.
  • 26. Are Batch and Streaming Systems Fundamentally Different? Both accumulate events from *T1...Tn, Tn+1...Tn+x,...* Or bucketed by Year-Month-Day-Hour? Streaming frameworks have schedulers
  • 27. A Unified Streaming Architecture Everything On The Streaming Platform Scala / Spark Streaming Mesos Akka Cassandra Kafka
  • 29. High Throughput Distributed Messaging High Scalability - billions of events per day Durability - no data loss Immutability Support Massive Number of Consumers Very ef cient and low latency Decouples Data Pipelines Automatic recovery from broker failures
  • 30. Stream Processing Simplified Kafka Streams In master, coming in v0.10 Removes the need to run another framework like Storm alongside Kafka Removes the need for separate infrastructures Common stream operations, e.g. join, lter, map, etc. Windowing Proper time modeling, e.g. event time vs. processing time Local state management with persistence and replication Schema and Avro support
  • 31. Spark Streaming Iterative ML, Interactive Querying, Graph, DataFrames One runtime for streaming and batch processing Join streaming and static data sets No code duplication Easy Kafka stream integration Easy to reconcile queries against multiple sources Easy integration of KV durable storage
  • 32. Apache Cassandra Horizontally scalable Multi-Region / Multi-Datacenter Always On - Survive regional outages Extremely fast writes: - perfect for ingestion of real time / machine data Very exible data modelling (lists, sets, custom data types) Easy to operate Best of breed storage technology, huge community BUT: Simple queries only OLTP-oriented/center
  • 33. High performance concurrency framework for Scala and Java Fault Tolerance Asynchronous messaging and data processing Parallelization Location Transparency Local / Remote Routing Akka: Cluster / Persistence / Streams
  • 34. Enables Streaming and Batch In One System Streaming ML and Analytics for Predictions In The Stream
  • 36. Spark Streaming Kafka Immutable Raw Data From Kafka Stream Replaying data streams: for fault tolerance, logic changes.. classKafkaStreamingActor(ssc:StreamingContext)extendsMyAggregationActor{
 valstream=KafkaUtils.createDirectStream(...)
.map(RawWeatherData(_)) stream .foreachRDD(_.toDF.write.format("filodb.spark") .option("dataset","rawdata").save()) /*Pre-Aggregatedatainthestreamforfastqueryingandaggregationlater.*/ stream.map(hour=> (hour.wsid,hour.year,hour.month,hour.day,hour.oneHourPrecip) ).saveToCassandra(timeseriesKeyspace,dailyPrecipTable)
 }
  • 37. Reading Data From Cassandra On Request, Further Aggregation Compute isolation in Akka Actor classTemperatureActor(sc:SparkContext)extendsAggregationActor{
 importakka.pattern.pipe defreceive:Actor.Receive={
 casee:GetMonthlyHiLowTemperature=>highLow(e,sender)
 }

 defhighLow(e:GetMonthlyHiLowTemperature,requester:ActorRef):Unit=
 sc.cassandraTable[DailyTemperature](timeseriesKeyspace,dailyTempAggregTable)
 .where("wsid=?ANDyear=?ANDmonth=?",e.wsid,e.year,e.month) .collectAsync()
 .map(MonthlyTemperature(_,e.wsid,e.year,e.month))pipeTorequester }
  • 38. Spark Streaming, MLLib Kafka, Cassandra valssc=newStreamingContext(sparkConf,Seconds(5)
) valtestData=ssc.cassandraTable[String](keyspace,table) .map(LabeledPoint.parse) valtrainingStream=KafkaUtils.createDirectStream[_,_,_,_](..) .map(transformFunc) .map(LabeledPoint.parse) trainingStream.saveToCassandra("ml_training_keyspace","raw_training_data")
 
valmodel=newStreamingLinearRegressionWithSGD()
 .setInitialWeights(Vectors.dense(weights))
 .trainOn(trainingStream) model .predictOnValues(testData.map(lp=>(lp.label,lp.features))) .saveToCassandra("ml_predictions_keyspace","predictions")
  • 39. What's Missing? One Pipeline For Fast + Big Data
  • 40. Using Cassandra for Batch Analytics / Event Storage / ML? Storage ef ciency and scan speeds for reading large volumes of data (for complex analytics, ML) become important concerns Regular Cassandra CQL tables are not very good at either storage ef ciency or scan speeds A different, analytics-optimized solution is needed...
  • 41. All hard work leads to pro t, but mere talk leads to poverty. - Proverbs 14:23
  • 42. Introducing FiloDB A distributed, versioned, columnar analytics database. Built for Streaming.   github.com/tuplejump/FiloDB
  • 43. Fast Analytics Storage Scan speeds competitive with Apache Parquet Up to 200x faster scan speeds than with Cassandra 2.x Flexible ltering along two dimensions Much more ef cient and exible partition key ltering Ef cient columnar storage, up to 40x more ef cient than Cassandra 2.x
  • 44. Comparing Storage Costs and Query Speeds https://www.oreilly.com/ideas/apache-cassandra-for-analytics- a-performance-and-storage-analysis
  • 45. Robust Distributed Storage Apache Cassandra as the rock-solid storage engine. Scale out with no SPOF. Cross-datacenter replication. Proven storage and database technology.
  • 46.   Cassandra-Like Data Model Column A Column B Partition key 1 Segment 1 Segment 2 Segment 1 Segment 2 Partition key 2 Segment 1 Segment 2 Segment 1 Segment 2 partition keys - distributes data around a cluster, and allows for ne grained and exible ltering segment keys - do range scans within a partition, e.g. by time slice primary key based ingestion and updates
  • 47. Flexible Filtering Unlike Cassandra, FiloDB offers very exible and ef cient ltering on partition keys. Partial key matches, fast IN queries on any part of the partition key. No need to write multiple tables to work around answering different queries.
  • 49. What's in the name? Rich sweet layers of distributed, versioned database goodness
  • 50. SNACK (SMACK) stack for all your Analytics Regular Cassandra tables for highly concurrent, aggregate / key-value lookups (dashboards) FiloDB + C* + Spark for ef cient long term event storage Ad hoc / SQL / BI Data source for MLLib / building models Data storage for classi ed / predicted / scored data
  • 51.
  • 52. Being Productionized as we speak... One enterprise with many TB of nancial and reporting data is moving their data warehouse to FiloDB + Cassandra + Spark Another startup uses FiloDB as event storage, feeds the events into Spark MLlib, scores incoming data, then stores the results back in FiloDB for low-latency use cases From their CTO: “I see close to MemSQL / Vertica or even better” “More cost effective than Redshift”
  • 53. FiloDB Use Cases Data Warehousing / BI < 10 second SLA, nontrivial reports, some concurrency need to store and query lots of data ef ciently Time series idempotent write API, simultaneous write and read workloads In-memory SQL web server 700 queries per second using in-memory column store
  • 54. FiloDB vs HDFS/Parquet FiloDB Parquet Ingestion Idempotent primary-key based; appends and replaces; deletes coming File-based append API only Filtering Partition-key and segment- key ltering Mostly le-based Scan speeds Parquet-like Good for OLAP Storage cost Within 35% of Parquet  
  • 55. FiloDB vs HDFS/Parquet In practice, with good data modeling, FiloDB is a far better t for low-latency / concurrent BI / reporting / dashboard applications.
  • 56. FiloDB vs Druid Different use cases: Druid is optimized mostly for OLAP cube / slice and dice analysis. Append only, keeps only aggregates, not a raw event store. FiloDB stores raw data - can be used to build ML models, visualize and analyze raw time series data, do complex event ow analysis - much more exible FiloDB can update/replace data FiloDB does not require data denormalization - can handle traditional BI star schemas with slowly changing dimension tables
  • 57. Come check out the demo! Visit FiloDB at the Developer Showcase (Expo hall) today!
  • 58. Machine Learning with Spark, Cassandra, and FiloDB
  • 59. Building a static model of NYC Taxi Trips Predict time to get to destination based on pickup point, time of day, other vars Need to read all data (full table scan)
  • 60. Dynamic models are better than static models Everything changes! Continuously re ne model based on recent streaming data + historical data + existing model
  • 62.
  • 63. The FiloDB Advantage for ML Able to update dynamic models based on massive data ow/updates Integrate historical and recent events to build models More data -> better models! Can store scored raw data / predictions back in FiloDB for fast user queries
  • 64. FiloDB - Roadmap Your input is appreciated! Productionization and automated stress testing Kafka input API / connector (without needing Spark) In-memory caching for signi cant query speedup True columnar querying and execution, using late materialization and vectorization techniques. GPU/SIMD. Projections. Often-repeated queries can be sped up signi cantly with projections.
  • 67. What are my storage needs? Non-persistent / in-memory: concurrent viewers Short term: latest trends Longer term: raw event and aggregate storage ML Models, predictions, scored data
  • 68. Spark RDDs Immutable, cache in memory and/or on disk Spark Streaming: UpdateStateByKey IndexedRDD - can update bits of data Snapshotting for recovery
  • 69. Using Cassandra for Short Term Storage 1020s 1010s 1000s Bus A Speed, GPS Bus B Bus C Primary key = (Bus UUID, timestamp) Easy queries: location and speed of single bus for a range of time Can also query most recent location + speed of all buses (slower)
  • 71. Scenarios BI Reporting, concurrency + seconds latency Ad-hoc queries Needing to do JOINs with fact tables + dimension tables Slowly changing dim tables / hard to denormalize Need to work with legacy BI tools
  • 72. Real-world DW Architecture Stack Ef cient columnar storage + ltering = low latency BI
  • 73. Modeling Fact Tables for FiloDB Single partition queries are really fast and take up only one thread Given the following two partition key columns: entity_number, year_month WHERE entity_number = '0453' AND year_month = '2014 December' Exact match for partition key is pushed down as one partition Consider the partition key carefully
  • 74. Cassandra often requires multiple tables What about the queries that do not translate to one partition? Cassandra has many restrictions on partition key ltering (as of 2.x). Table 1: partition key = (entity_number, year_month) Can push down: WHERE entity_number = NN AND year_month IN ('2014 Jan', '2014 Feb')as well as equals Table 2: partition key = (year_month, entity_number) Can push down: WHERE year_month = YYMM AND entity_number IN (123, 456)as well as equals IN clause must be the last column to be pushed down. Two tables are needed just for ef cient IN queries on either entity_number or year_month.
  • 75. FiloDB Flexible Partition Filters = WIN With ONE table, FiloDB offers FAST, arbitrary partition key ltering. All of the below are pushed down: WHERE year_month IN ('2014 Jan', '2014 Feb') (all entities) WHERE entity_number = 146(all year months) Any combo of =, IN Space savings: 27 *2 = 54x
  • 76. Multi-Table JOINs with just Cassandra
  • 78. Sub-second Multi-Table JOINs with FiloDB Four tables, all of them single-partition queries Two tables were switched from regular Cassandra tables to FiloDB tables. 40-60 columns each, ~60k items in partition. Scan times went down from 5-6 seconds to < 250ms For more details, please see this .Planet Cassandra blog post
  • 79. Scalable Time-Series / Event Storage with FiloDB
  • 80. Designed for Streaming New rows appended via Spark Streaming or Kafka Writes are idempotent - easy exactly once ingestion Converted to columnar chunks on ingest and stored in C* FiloDB keeps your data sorted as it is being ingested
  • 81. Spark Streaming -> FiloDB valratingsStream=KafkaUtils.createDirectStream[String,String,StringDecoder,Strin ratingsStream.foreachRDD{ (message:RDD[(String,String)],batchTime:Time)=>{ valdf=message.map(_._2.split(",")).map(rating=>Rating(rating(0).trim.toInt,r toDF("fromuserid","touserid","rating") //addthebatchtimetotheDataFrame valdfWithBatchTime=df.withColumn("batch_time",org.apache.spark.sql.functions.l //savetheDataFrametoFiloDB dfWithBatchTime.write.format("filodb.spark") .option("dataset","ratings") .save() } } One-line change to write to FiloDB vs Cassandra
  • 82. Modeling example: NYC Taxi Dataset The public contains telemetry (pickup, dropoff locations, times) info on millions of taxi rides in NYC. NYC Taxi Dataset Medallion pre x 1/1 - 1/6 1/7 - 1/12 AA records records AB records records Partition key - :stringPrefix medallion 2- hash multiple drivers trips into ~300 partitions Segment key - :timeslice pickup_datetime 6d Row key - hack_license, pickup_datetime Allows for easy ltering by individual drivers, and slicing by time.
  • 83. DEMO TIME New York City Taxi Data Demo (Spark Notebook) To follow along: https://github.com/tuplejump/FiloDB/blob/master/doc/FiloDB_Taxi_G
  • 84. Fast, Updatable In-Memory Columnar Storage Unlike RDDs and DataFrames, FiloDB can ingest new data, and still be fast Unlike RDDs, FiloDB can lter in multiple ways, no need for entire table scan FAIR scheduler + sub-second latencies => web speed queries
  • 85. 700 Queries Per Second in Apache Spark! Even for datasets with 15 million rows! Using FiloDB's InMemoryColumnStore, single host / MBP, 5GB RAM SQL to DataFrame caching For more details, see .this blog post
  • 87. Multiple ways to Accelerate Queries Columnar projection - read fewer columns, saves I/O Partition key ltering - read less data Sort key / PK ltering - read from subset of keys Possible because FiloDB keeps data sorted Versioning - write to multiple versions, read from the one you choose
  • 88. Cassandra CQL vs Columnar Layout Cassandra stores CQL tables row-major, each row spans multiple cells: PartitionKey 01: rst 01:last 01:age 02: rst 02:last 02:age Sales Bob Jones 34 Susan O'Connor 40 Engineering Dilbert P ? Dogbert Dog 1   Columnar layouts are column-major: PartitionKey rst last age Sales Bob, Susan Jones, O'Connor 34, 40 Engineering Dilbert, Dogbert P, Dog ?, 1
  • 90. FiloDB Architecture ColumnStore API - currently Cassandra and InMemory, you can implement other backends - ElasticSearch? etc.