SlideShare a Scribd company logo
1 of 57
Download to read offline
Martin Zapletal @zapletal_martin
Cake Solutions @cakesolutions
#CassandraSummit
Presented by Anirvan Chakraborty @anirvan_c
● Introduction
● Event sourcing and CQRS
● An emerging technology stack to handle data
● A reference application and it’s architecture
● A few use cases of the reference application
● Conclusion
● Increasing importance of data analytics
● Current state
○ Destructive updates
○ Analytics tools with poor scalability and integration
○ Manual processes
○ Slow iterations
○ Not suitable for large amounts of data
● Whole lifecycle of data
● Data processing
● Data stores
● Integration and messaging
● Distributed computing primitives
● Cluster managers and task schedulers
● Deployment, configuration management and DevOps
● Data analytics and machine learning
● Spark, Mesos, Akka, Cassandra, Kafka (SMACK, Infinity)
ACID Mutable State
● Create, Read, Update, Delete
● Exposes mutable internal state
● Many read methods on repositories
● Mapping of data model and objects (impedance mismatch)
● No auditing
● No separation of concerns (read / write, command / event)
● Strongly consistent
● Difficult optimizations of reads / writes
● Difficult to scale
● Intent, behaviour, history, is lost
Balance = 5
Balance = 10
Update
Account
Balance = 10
Account
[1]
CQRS
Client
QueryCommand
DBDB
Denormalise
/Precompute
Kappa architecture
Batch-Pipeline
Kafka
Allyourdata
NoSQL
SQL
Spark
Client
Client
Client Views
Stream
processor
Flume
Scoop
Hive
Impala
Oozie
HDFS
Lambda Architecture
Batch Layer Serving
Layer
Stream layer (fast)
Query
Query
Allyourdata Serving DB
[2, 3]
● Append only data store
● No updates or deletes (rewriting history)
● Immutable data model
● Decouples data model of the application and storage
● Current state not persisted, but derived. A sequence of updates that led to it.
● History, state known at any point in time
● Replayable
● Source of truth
● Optimisations possible
● Works well in distributed environment - easy partitioning, conflicts
● Helps avoiding transactions
● Works well with DDD
userId date change
1
1
1
10/10/2015
11/10/2015
23/10/2015
+300
-100
-200
1 24/10/2015 +100
balanceChanged
event
balanceChanged
balanceChanged
balanceChanged
Event journal
● Command Query Responsibility Segregation
● Read and write logically and physically separated
● Reasoning about the application
● Clear separation of concerns (business logic)
● Often different technology, scalability
● Often lower consistency - eventual, causal
Command
● Write side
● Messages, requests to mutate state
● Behaviour, serialized method call essentially
● Don’t expose state
● Validated and may be rejected or emit one or more events (e.g. submitting a form)
Event
● Write side
● Immutable
● Indicating something that has happened
● Atomic record of state change
● Audit log
Query
● Read side
● Precomputed
userId = 1
updateBalance
(+100)
Write
Command
Event
userId date change
1
1
1
10/10/2015
11/10/2015
23/10/2015
+300
-100
-200
1 24/10/2015 +100
balance
Changed
event
balance
Changed
balance
Changed
balance
Changed
Event journal
Command
handler
Read
balance
1 100
userId = 1
balance = 100
Query
userId
● Partial order of events for each entity
● Operation semantics, CRDTs
UserNameUpdated(B)
UserNameUpdated(B)
UserNameUpdated(A)
UserNameUpdated(A)
● Localization
● Conflicting concurrent histories
○ Resubmission
○ Deduplication
○ Replication
● Identifier
● Version
● Timestamp
● Vector clock
● Actor framework for truly concurrent and distributed systems
● Thread safe mutable state - consistency boundary
● Domain modelling, distributed state
● Simple programming model - asynchronously send messages, create
new actors, change behaviour
● Supports CQRS/ES
● Fully distributed - asynchronous, delivery guarantees, failures, time
and order, consistency, availability, communication patterns, data
locality, persistence, durability, concurrent updates, conflicts,
divergence, invariants, ...
?
?
? + 1
? + 1
? + 2
UserId = 1
Name = Bob
BankAccountId = 1
Balance = 1000
UserId = 1
Name = Alice
● Distributed domain modelling
● In memory
● Ordering, consistency
id = 1
● Actor backed by data store
● Immutable event sourced journal
● Supports CQRS (write and read side)
● Persistence, replay on failure, rebalance, at least once delivery
user1, event 2
user1, event 3
user1, event 4
user1, event 1
class UserActor extends PersistentActor {
override def persistenceId: String = UserPersistenceId(self.path.name).persistenceId
override def receiveCommand: Receive = notRegistered(DistributedData(context.system).replicator)
def notRegistered(distributedData: ActorRef): Receive = {
case cmd: AccountCommand =>
persist(AccountEvent(cmd.account)){ acc =>
context.become(registered(acc))
sender() ! /-()
}
}
def registered(account: Account): Receive = {
case eres @ EntireResistanceExerciseSession(id, session, sets, examples, deviations) =>
persist(eres)(data => sender() ! /-(id))
}
override def receiveRecover: Receive = {
...
}
}
● Akka Persistence Cassandra journal
○ Globally distributed journal
○ Scalable, resilient, highly available
○ Performant, operational database
● Community plugins
akka {
persistence {
journal.plugin = "cassandra-journal"
snapshot-store.plugin = "cassandra-snapshot-store"
}
}
● Partition-size
● Events in each cluster partition ordered (persistenceId - partition pair)
CREATE TABLE IF NOT EXISTS ${tableName} (
processor_id text,
partition_nr bigint,
sequence_nr bigint,
marker text,
message blob,
PRIMARY KEY ((processor_id, partition_nr),
sequence_nr, marker))
WITH COMPACT STORAGE
AND gc_grace_seconds = ${config.
gc_grace_seconds}
processor_id partition_nr sequence_nr marker message
user-1 0 0 H 0x0a6643b334...
user-1 0 1 A 0x0ab2020801...
user-1 0 2 A 0x0a98020801...
● Internal state, moment in time
● Read optimization
CREATE TABLE IF NOT EXISTS ${tableName} (
processor_id text,
sequence_nr bigint,
timestamp bigint,
snapshot blob,
PRIMARY KEY (processor_id, sequence_nr))
WITH CLUSTERING ORDER BY (sequence_nr DESC)
processor_id sequence_nr snapshot timestamp
user-1 16 0x0400000001... 1441696908210
user-1 20 0x0400000001... 1441697587765
● Uses Akka serialization
0x0a6643b334 …
PersistentRepr
Akka.Serialization
Payload: T
Protobuff
actor {
serialization-bindings {
"io.muvr.exercise.ExercisePlanDeviation" = kryo,
"io.muvr.exercise.ResistanceExercise" = kryo,
}
serializers {
java = "akka.serialization.JavaSerializer"
kryo = "com.twitter.chill.akka.AkkaSerializer"
}
}
class UserActorView(userId: String) extends PersistentView {
override def persistenceId: String = UserPersistenceId(userId).persistenceId
override def viewId: String = UserPersistenceId(userId).persistentViewId
override def autoUpdateInterval: FiniteDuration = FiniteDuration(100, TimeUnit.MILLISECONDS)
def receive: Receive = viewState(List.empty)
def viewState(processedDeviations: List[ExercisePlanProcessedDeviation]): Receive = {
case EntireResistanceExerciseSession(_, _, _, _, deviations) if isPersistent =>
context.become(viewState(deviations.filter(condition).map(process) ::: processedDeviations))
case GetProcessedDeviations => sender() ! processedDeviations
}
}
● Akka 2.4
● Potentially infinite stream of data
● Ordered, replayable, resumable
● Aggregation, transformation, moving data
● EventsByPersistenceId
● AllPersistenceids
● EventsByTag
val readJournal =
PersistenceQuery(system).readJournalFor(CassandraJournal.Identifier)
val source = readJournal.query(
EventsByPersistenceId(UserPersistenceId(name).persistenceId, 0, Long.MaxValue), NoRefresh)
.map(_.event)
.collect{ case s: EntireResistanceExerciseSession => s }
.mapConcat(_.deviations)
.filter(condition)
.map(process)
implicit val mat = ActorMaterializer()
val result = source.runFold(List.empty[ExercisePlanDeviation])((x, y) => y :: x)
● Potentially infinite stream of events
Source[Any].map(process).filter(condition)
Publisher Subscriber
process
condition
backpressure
● In Akka we have the read and write sides separated,
in Cassandra we don’t
● Different data model
● Avoid using operational datastore
● Eventual consistency
● Streaming transformations to different format
● Unify journalled and other data
● Computations and analytics queries on the data
● Often iterative, complex, expensive computations
● Prepared and interactive queries
● Data from multiple sources, joins and transformations
● Often directly on a stream of data
● Whole history of events
● Historical behaviour
● Works retrospectively, can answer questions in the future that we don’t
know exist yet
● Various data types from various sources
● Large amounts of fast data
● Automated analytics
● Cassandra 3.0 - user defined functions, functional indexes, aggregation
functions, materialized views
● Server side denormalization
● Eventual consistency
● Copy of data with different partitioning
userId
performance
● In memory dataflow distributed data processing framework, streaming
and batch
● Distributes computation using a higher level API
● Load balancing
● Moves computation to data
● Fault tolerant
● Resilient Distributed Datasets
● Fault tolerance
● Caching
● Serialization
● Transformations
○ Lazy, form the DAG
○ map, filter, flatMap, union, group, reduce, sort, join, repartition, cartesian, glom, ...
● Actions
○ Execute DAG, retrieve result
○ reduce, collect, count, first, take, foreach, saveAs…, min, max, ...
● Accumulators
● Broadcast Variables
● Integration
● Streaming
● Machine Learning
● Graph Processing
textFile mapmap
reduceByKey
collect
sc.textFile("counts")
.map(line => line.split("t"))
.map(word => (word(0), word(1).toInt))
.reduceByKey(_ + _)
.collect()
[4]
Spark master
Spark worker
Cassandra
● Cassandra can store
● Spark can process
● Gathering large amounts of heterogeneous data
● Queries
● Transformations
● Complex computations
● Machine learning, data mining, analytics
● Now possible
● Prepared and interactive queries
lazy val sparkConf: SparkConf =
new SparkConf()
.setAppName(...).setMaster(...).set("spark.cassandra.connection.host", "127.0.0.1")
val sc = new SparkContext(sparkConf)
val data = sc.cassandraTable[T]("keyspace", "table").select("columns")
val processedData = data.flatMap(...)...
processedData.saveToCassandra("keyspace", "table")
● Akka Analytics project
● Handles custom Akka serialization
case class JournalKey(persistenceId: String, partition: Long, sequenceNr: Long)
lazy val sparkConf: SparkConf =
new SparkConf()
.setAppName(...).setMaster(...).set("spark.cassandra.connection.host", "127.0.0.1")
val sc = new SparkContext(sparkConf)
val events: RDD[(JournalKey, Any)] = sc.eventTable()
events.sortByKey().map(...).filter(...).collect().foreach(println)
● Spark streaming
● Precomputing using spark or replication often aiming for different data
model
Operational cluster Analytics cluster
Precomputation /
replication
Integration with
other data sources
val events: RDD[(JournalKey, Any)] = sc.eventTable().cache().filterClass[EntireResistanceExerciseSession].flatMap(_.deviations)
val deviationsFrequency = sqlContext.sql(
"""SELECT planned.exercise, hour(time), COUNT(1)
FROM exerciseDeviations
WHERE planned.exercise = 'bench press'
GROUP BY planned.exercise, hour(time)""")
val deviationsFrequency2 = exerciseDeviationsDF
.where(exerciseDeviationsDF("planned.exercise") === "bench press")
.groupBy(
exerciseDeviationsDF("planned.exercise"),
exerciseDeviationsDF("time”))
.count()
val deviationsFrequency3 = exerciseDeviations
.filter(_.planned.exercise == "bench press")
.groupBy(d => (d.planned.exercise, d.time.getHours))
.map(d => (d._1, d._2.size))
def toVector(user: User): mllib.linalg.Vector =
Vectors.dense(
user.frequency, user.performanceIndex, user.improvementIndex)
val events: RDD[(JournalKey, Any)] = sc.eventTable().cache()
val users: RDD[User] = events.filterClass[User]
val kmeans = new KMeans()
.setK(5)
.set...
val clusters = kmeans.run(users.map(_.toVector))
val weight: RDD[(JournalKey, Any)] = sc.eventTable().cache()
val exerciseDeviations = events
.filterClass[EntireResistanceExerciseSession]
.flatMap(session =>
session.sets.flatMap(set =>
set.sets.map(exercise => (session.id.id, exercise.exercise))))
.groupBy(e => e)
.map(g =>
Rating(normalize(g._1._1), normalize(g._1._2),
normalize(g._2.size)))
val model = new ALS().run(ratings)
val predictions = model.predict(recommend)
bench
press
bicep
curl
dead
lift
user 1 5 2
user 2 4 3
user 3 5 2
user 4 3 1
val events = sc.eventTable().cache().toDF()
val lr = new LinearRegression()
val pipeline = new Pipeline().setStages(Array(new UserFilter(), new ZScoreNormalizer(),
new IntensityFeatureExtractor(), lr))
val paramGrid = new ParamGridBuilder()
.addGrid(lr.regParam, Array(0.1, 0.01))
.addGrid(lr.fitIntercept, Array(true, false))
getEligibleUsers(events, sessionEndedBefore)
.map { user =>
val trainValidationSplit = new TrainValidationSplit()
.setEstimator(pipeline)
.setEvaluator(new RegressionEvaluator)
.setEstimatorParamMaps(paramGrid)
val model = trainValidationSplit.fit(
events,
ParamMap(ParamPair(userIdParam, user)))
val testData = // Prepare test data.
val predictions = model.transform(testData)
submitResult(userId, predictions, config)
}
val events: RDD[(JournalKey, Any)] = sc.eventTable().cache()
val connections = events.filterClass[Connections]
val vertices: RDD[(VertexId, Long)] =
connections.map(c => (c.id, 1l))
val edges: RDD[Edge[Long]] = connections
.flatMap(c => c.connections
.map(Edge(c.id, _, 1l)))
val graph = Graph(vertices, edges)
val ranks = graph.pageRank(0.0001).vertices
7 * Dumbbell
Alternating Curl
Data
Data
Preprocessing
Preprocessing
Features
Features
Training
Testing
Error %
● Exercise domain as an example
● Analytics of both batch (offline) and streaming (online) data
● Analytics important in other areas (banking, stock market, network,
cluster monitoring, business intelligence, commerce, internet of things, ...)
● Enabling value of data
● Event sourcing
● CQRS
● Technologies to handle the data
○ Spark
○ Mesos
○ Akka
○ Cassandra
○ Kafka
● Handling data
● Insights and analytics enable value in data
● Jobs at www.cakesolutions.net/careers
● Code at https://github.com/muvr
● Martin Zapletal @zapletal_martin
● Anirvan Chakraborty @anirvan_c
[1] http://www.benstopford.com/2015/04/28/elements-of-scale-composing-and-scaling-data-platforms/
[2] http://malteschwarzkopf.de/research/assets/google-stack.pdf
[3] http://malteschwarzkopf.de/research/assets/facebook-stack.pdf
[4] http://www.slideshare.net/LisaHua/spark-overview-37479609

More Related Content

What's hot

Building RightScale's Globally Distributed Datastore - RightScale Compute 2013
Building RightScale's Globally Distributed Datastore - RightScale Compute 2013Building RightScale's Globally Distributed Datastore - RightScale Compute 2013
Building RightScale's Globally Distributed Datastore - RightScale Compute 2013RightScale
 
Cassandra Data Maintenance with Spark
Cassandra Data Maintenance with SparkCassandra Data Maintenance with Spark
Cassandra Data Maintenance with SparkDataStax Academy
 
The dr overnight dba
The dr overnight dbaThe dr overnight dba
The dr overnight dbagdabate
 
Sql server performance tuning
Sql server performance tuningSql server performance tuning
Sql server performance tuningJugal Shah
 
Ledingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @LendingkartLedingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @LendingkartMukesh Singh
 
Got documents - The Raven Bouns Edition
Got documents - The Raven Bouns EditionGot documents - The Raven Bouns Edition
Got documents - The Raven Bouns EditionMaggie Pint
 
Oracle Query Optimizer - An Introduction
Oracle Query Optimizer - An IntroductionOracle Query Optimizer - An Introduction
Oracle Query Optimizer - An Introductionadryanbub
 
An introduction to the WSO2 Analytics Platform
An introduction to the WSO2 Analytics Platform   An introduction to the WSO2 Analytics Platform
An introduction to the WSO2 Analytics Platform Sriskandarajah Suhothayan
 
BigDataSpain 2016: Introduction to Apache Apex
BigDataSpain 2016: Introduction to Apache ApexBigDataSpain 2016: Introduction to Apache Apex
BigDataSpain 2016: Introduction to Apache ApexThomas Weise
 

What's hot (11)

Building RightScale's Globally Distributed Datastore - RightScale Compute 2013
Building RightScale's Globally Distributed Datastore - RightScale Compute 2013Building RightScale's Globally Distributed Datastore - RightScale Compute 2013
Building RightScale's Globally Distributed Datastore - RightScale Compute 2013
 
Cassandra Data Maintenance with Spark
Cassandra Data Maintenance with SparkCassandra Data Maintenance with Spark
Cassandra Data Maintenance with Spark
 
The dr overnight dba
The dr overnight dbaThe dr overnight dba
The dr overnight dba
 
Sql server performance tuning
Sql server performance tuningSql server performance tuning
Sql server performance tuning
 
Ledingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @LendingkartLedingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @Lendingkart
 
Got documents - The Raven Bouns Edition
Got documents - The Raven Bouns EditionGot documents - The Raven Bouns Edition
Got documents - The Raven Bouns Edition
 
Oracle Query Optimizer - An Introduction
Oracle Query Optimizer - An IntroductionOracle Query Optimizer - An Introduction
Oracle Query Optimizer - An Introduction
 
It Depends
It DependsIt Depends
It Depends
 
Druid
DruidDruid
Druid
 
An introduction to the WSO2 Analytics Platform
An introduction to the WSO2 Analytics Platform   An introduction to the WSO2 Analytics Platform
An introduction to the WSO2 Analytics Platform
 
BigDataSpain 2016: Introduction to Apache Apex
BigDataSpain 2016: Introduction to Apache ApexBigDataSpain 2016: Introduction to Apache Apex
BigDataSpain 2016: Introduction to Apache Apex
 

Viewers also liked

Scaling wix with microservices architecture jax london-2015
Scaling wix with microservices architecture jax london-2015Scaling wix with microservices architecture jax london-2015
Scaling wix with microservices architecture jax london-2015Aviran Mordo
 
GC Tuning Confessions Of A Performance Engineer - Improved :)
GC Tuning Confessions Of A Performance Engineer - Improved :)GC Tuning Confessions Of A Performance Engineer - Improved :)
GC Tuning Confessions Of A Performance Engineer - Improved :)Monica Beckwith
 
Java micro-services
Java micro-servicesJava micro-services
Java micro-servicesJames Lewis
 
Apache Cassandra for Timeseries- and Graph-Data
Apache Cassandra for Timeseries- and Graph-DataApache Cassandra for Timeseries- and Graph-Data
Apache Cassandra for Timeseries- and Graph-DataGuido Schmutz
 
Akka persistence == event sourcing in 30 minutes
Akka persistence == event sourcing in 30 minutesAkka persistence == event sourcing in 30 minutes
Akka persistence == event sourcing in 30 minutesKonrad Malawski
 
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...Lightbend
 

Viewers also liked (6)

Scaling wix with microservices architecture jax london-2015
Scaling wix with microservices architecture jax london-2015Scaling wix with microservices architecture jax london-2015
Scaling wix with microservices architecture jax london-2015
 
GC Tuning Confessions Of A Performance Engineer - Improved :)
GC Tuning Confessions Of A Performance Engineer - Improved :)GC Tuning Confessions Of A Performance Engineer - Improved :)
GC Tuning Confessions Of A Performance Engineer - Improved :)
 
Java micro-services
Java micro-servicesJava micro-services
Java micro-services
 
Apache Cassandra for Timeseries- and Graph-Data
Apache Cassandra for Timeseries- and Graph-DataApache Cassandra for Timeseries- and Graph-Data
Apache Cassandra for Timeseries- and Graph-Data
 
Akka persistence == event sourcing in 30 minutes
Akka persistence == event sourcing in 30 minutesAkka persistence == event sourcing in 30 minutes
Akka persistence == event sourcing in 30 minutes
 
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...
Building Streaming And Fast Data Applications With Spark, Mesos, Akka, Cassan...
 

Similar to Cassandra as event sourced journal for big data analytics

Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...
Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...
Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...Martin Zapletal
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseBig Data Spain
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...Amazon Web Services
 
Scalability truths and serverless architectures
Scalability truths and serverless architecturesScalability truths and serverless architectures
Scalability truths and serverless architecturesRegunath B
 
Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flinkRenato Guimaraes
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analyticsXiang Fu
 
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
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache SparkLucian Neghina
 
Challenges in Building a Data Pipeline
Challenges in Building a Data PipelineChallenges in Building a Data Pipeline
Challenges in Building a Data PipelineManish Kumar
 
Challenges in building a Data Pipeline
Challenges in building a Data PipelineChallenges in building a Data Pipeline
Challenges in building a Data PipelineHevo Data Inc.
 
An adaptive and eventually self healing framework for geo-distributed real-ti...
An adaptive and eventually self healing framework for geo-distributed real-ti...An adaptive and eventually self healing framework for geo-distributed real-ti...
An adaptive and eventually self healing framework for geo-distributed real-ti...Angad Singh
 
Serverless Event Streaming with Pulsar Functions
Serverless Event Streaming with Pulsar FunctionsServerless Event Streaming with Pulsar Functions
Serverless Event Streaming with Pulsar FunctionsStreamNative
 
M|18 Analytics as a Service
M|18 Analytics as a ServiceM|18 Analytics as a Service
M|18 Analytics as a ServiceMariaDB plc
 
Will it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsWill it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsNavina Ramesh
 
Fluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconFluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconN Masahiro
 
Scala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache sparkScala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache sparkDemi Ben-Ari
 
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...Databricks
 
Hoodie: How (And Why) We built an analytical datastore on Spark
Hoodie: How (And Why) We built an analytical datastore on SparkHoodie: How (And Why) We built an analytical datastore on Spark
Hoodie: How (And Why) We built an analytical datastore on SparkVinoth Chandar
 

Similar to Cassandra as event sourced journal for big data analytics (20)

Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...
Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...
Large volume data analysis on the Typesafe Reactive Platform - Big Data Scala...
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas Weise
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
 
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...
How Netflix Uses Amazon Kinesis Streams to Monitor and Optimize Large-scale N...
 
Scalability truths and serverless architectures
Scalability truths and serverless architecturesScalability truths and serverless architectures
Scalability truths and serverless architectures
 
AmazonRedshift
AmazonRedshiftAmazonRedshift
AmazonRedshift
 
Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flink
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analytics
 
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
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache Spark
 
Challenges in Building a Data Pipeline
Challenges in Building a Data PipelineChallenges in Building a Data Pipeline
Challenges in Building a Data Pipeline
 
Challenges in building a Data Pipeline
Challenges in building a Data PipelineChallenges in building a Data Pipeline
Challenges in building a Data Pipeline
 
An adaptive and eventually self healing framework for geo-distributed real-ti...
An adaptive and eventually self healing framework for geo-distributed real-ti...An adaptive and eventually self healing framework for geo-distributed real-ti...
An adaptive and eventually self healing framework for geo-distributed real-ti...
 
Serverless Event Streaming with Pulsar Functions
Serverless Event Streaming with Pulsar FunctionsServerless Event Streaming with Pulsar Functions
Serverless Event Streaming with Pulsar Functions
 
M|18 Analytics as a Service
M|18 Analytics as a ServiceM|18 Analytics as a Service
M|18 Analytics as a Service
 
Will it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsWill it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing Applications
 
Fluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at KubeconFluentd and Distributed Logging at Kubecon
Fluentd and Distributed Logging at Kubecon
 
Scala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache sparkScala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache spark
 
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...
Incremental Processing on Large Analytical Datasets with Prasanna Rajaperumal...
 
Hoodie: How (And Why) We built an analytical datastore on Spark
Hoodie: How (And Why) We built an analytical datastore on SparkHoodie: How (And Why) We built an analytical datastore on Spark
Hoodie: How (And Why) We built an analytical datastore on Spark
 

Recently uploaded

Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...
Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...
Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...Sapana Sha
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfLars Albertsson
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝soniya singh
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024thyngster
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingNeil Barnes
 
Data Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxData Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxFurkanTasci3
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPramod Kumar Srivastava
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts ServiceSapana Sha
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationshipsccctableauusergroup
 
Call Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceCall Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceSapana Sha
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)jennyeacort
 
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort servicejennyeacort
 
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理e4aez8ss
 
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...Florian Roscheck
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhijennyeacort
 
GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]📊 Markus Baersch
 

Recently uploaded (20)

Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...
Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...
Saket, (-DELHI )+91-9654467111-(=)CHEAP Call Girls in Escorts Service Saket C...
 
Call Girls in Saket 99530🔝 56974 Escort Service
Call Girls in Saket 99530🔝 56974 Escort ServiceCall Girls in Saket 99530🔝 56974 Escort Service
Call Girls in Saket 99530🔝 56974 Escort Service
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdf
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data Storytelling
 
E-Commerce Order PredictionShraddha Kamble.pptx
E-Commerce Order PredictionShraddha Kamble.pptxE-Commerce Order PredictionShraddha Kamble.pptx
E-Commerce Order PredictionShraddha Kamble.pptx
 
Data Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxData Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptx
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts Service
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships
 
Call Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts ServiceCall Girls In Dwarka 9654467111 Escorts Service
Call Girls In Dwarka 9654467111 Escorts Service
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
 
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
 
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
科罗拉多大学波尔得分校毕业证学位证成绩单-可办理
 
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
 
GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]
 

Cassandra as event sourced journal for big data analytics

  • 1. Martin Zapletal @zapletal_martin Cake Solutions @cakesolutions #CassandraSummit Presented by Anirvan Chakraborty @anirvan_c
  • 2. ● Introduction ● Event sourcing and CQRS ● An emerging technology stack to handle data ● A reference application and it’s architecture ● A few use cases of the reference application ● Conclusion
  • 3. ● Increasing importance of data analytics ● Current state ○ Destructive updates ○ Analytics tools with poor scalability and integration ○ Manual processes ○ Slow iterations ○ Not suitable for large amounts of data
  • 4. ● Whole lifecycle of data ● Data processing ● Data stores ● Integration and messaging ● Distributed computing primitives ● Cluster managers and task schedulers ● Deployment, configuration management and DevOps ● Data analytics and machine learning ● Spark, Mesos, Akka, Cassandra, Kafka (SMACK, Infinity)
  • 6. ● Create, Read, Update, Delete ● Exposes mutable internal state ● Many read methods on repositories ● Mapping of data model and objects (impedance mismatch) ● No auditing ● No separation of concerns (read / write, command / event) ● Strongly consistent ● Difficult optimizations of reads / writes ● Difficult to scale ● Intent, behaviour, history, is lost Balance = 5 Balance = 10 Update Account Balance = 10 Account
  • 9. ● Append only data store ● No updates or deletes (rewriting history) ● Immutable data model ● Decouples data model of the application and storage ● Current state not persisted, but derived. A sequence of updates that led to it. ● History, state known at any point in time ● Replayable ● Source of truth ● Optimisations possible ● Works well in distributed environment - easy partitioning, conflicts ● Helps avoiding transactions ● Works well with DDD
  • 10. userId date change 1 1 1 10/10/2015 11/10/2015 23/10/2015 +300 -100 -200 1 24/10/2015 +100 balanceChanged event balanceChanged balanceChanged balanceChanged Event journal
  • 11. ● Command Query Responsibility Segregation ● Read and write logically and physically separated ● Reasoning about the application ● Clear separation of concerns (business logic) ● Often different technology, scalability ● Often lower consistency - eventual, causal
  • 12. Command ● Write side ● Messages, requests to mutate state ● Behaviour, serialized method call essentially ● Don’t expose state ● Validated and may be rejected or emit one or more events (e.g. submitting a form) Event ● Write side ● Immutable ● Indicating something that has happened ● Atomic record of state change ● Audit log Query ● Read side ● Precomputed
  • 13. userId = 1 updateBalance (+100) Write Command Event userId date change 1 1 1 10/10/2015 11/10/2015 23/10/2015 +300 -100 -200 1 24/10/2015 +100 balance Changed event balance Changed balance Changed balance Changed Event journal Command handler Read balance 1 100 userId = 1 balance = 100 Query userId
  • 14. ● Partial order of events for each entity ● Operation semantics, CRDTs UserNameUpdated(B) UserNameUpdated(B) UserNameUpdated(A) UserNameUpdated(A)
  • 15. ● Localization ● Conflicting concurrent histories ○ Resubmission ○ Deduplication ○ Replication ● Identifier ● Version ● Timestamp ● Vector clock
  • 16.
  • 17.
  • 18. ● Actor framework for truly concurrent and distributed systems ● Thread safe mutable state - consistency boundary ● Domain modelling, distributed state ● Simple programming model - asynchronously send messages, create new actors, change behaviour ● Supports CQRS/ES ● Fully distributed - asynchronous, delivery guarantees, failures, time and order, consistency, availability, communication patterns, data locality, persistence, durability, concurrent updates, conflicts, divergence, invariants, ...
  • 19. ? ? ? + 1 ? + 1 ? + 2 UserId = 1 Name = Bob BankAccountId = 1 Balance = 1000 UserId = 1 Name = Alice
  • 20. ● Distributed domain modelling ● In memory ● Ordering, consistency id = 1
  • 21. ● Actor backed by data store ● Immutable event sourced journal ● Supports CQRS (write and read side) ● Persistence, replay on failure, rebalance, at least once delivery
  • 22. user1, event 2 user1, event 3 user1, event 4 user1, event 1
  • 23. class UserActor extends PersistentActor { override def persistenceId: String = UserPersistenceId(self.path.name).persistenceId override def receiveCommand: Receive = notRegistered(DistributedData(context.system).replicator) def notRegistered(distributedData: ActorRef): Receive = { case cmd: AccountCommand => persist(AccountEvent(cmd.account)){ acc => context.become(registered(acc)) sender() ! /-() } } def registered(account: Account): Receive = { case eres @ EntireResistanceExerciseSession(id, session, sets, examples, deviations) => persist(eres)(data => sender() ! /-(id)) } override def receiveRecover: Receive = { ... } }
  • 24. ● Akka Persistence Cassandra journal ○ Globally distributed journal ○ Scalable, resilient, highly available ○ Performant, operational database ● Community plugins akka { persistence { journal.plugin = "cassandra-journal" snapshot-store.plugin = "cassandra-snapshot-store" } }
  • 25. ● Partition-size ● Events in each cluster partition ordered (persistenceId - partition pair) CREATE TABLE IF NOT EXISTS ${tableName} ( processor_id text, partition_nr bigint, sequence_nr bigint, marker text, message blob, PRIMARY KEY ((processor_id, partition_nr), sequence_nr, marker)) WITH COMPACT STORAGE AND gc_grace_seconds = ${config. gc_grace_seconds} processor_id partition_nr sequence_nr marker message user-1 0 0 H 0x0a6643b334... user-1 0 1 A 0x0ab2020801... user-1 0 2 A 0x0a98020801...
  • 26. ● Internal state, moment in time ● Read optimization CREATE TABLE IF NOT EXISTS ${tableName} ( processor_id text, sequence_nr bigint, timestamp bigint, snapshot blob, PRIMARY KEY (processor_id, sequence_nr)) WITH CLUSTERING ORDER BY (sequence_nr DESC) processor_id sequence_nr snapshot timestamp user-1 16 0x0400000001... 1441696908210 user-1 20 0x0400000001... 1441697587765
  • 27. ● Uses Akka serialization 0x0a6643b334 … PersistentRepr Akka.Serialization Payload: T Protobuff actor { serialization-bindings { "io.muvr.exercise.ExercisePlanDeviation" = kryo, "io.muvr.exercise.ResistanceExercise" = kryo, } serializers { java = "akka.serialization.JavaSerializer" kryo = "com.twitter.chill.akka.AkkaSerializer" } }
  • 28. class UserActorView(userId: String) extends PersistentView { override def persistenceId: String = UserPersistenceId(userId).persistenceId override def viewId: String = UserPersistenceId(userId).persistentViewId override def autoUpdateInterval: FiniteDuration = FiniteDuration(100, TimeUnit.MILLISECONDS) def receive: Receive = viewState(List.empty) def viewState(processedDeviations: List[ExercisePlanProcessedDeviation]): Receive = { case EntireResistanceExerciseSession(_, _, _, _, deviations) if isPersistent => context.become(viewState(deviations.filter(condition).map(process) ::: processedDeviations)) case GetProcessedDeviations => sender() ! processedDeviations } }
  • 29. ● Akka 2.4 ● Potentially infinite stream of data ● Ordered, replayable, resumable ● Aggregation, transformation, moving data ● EventsByPersistenceId ● AllPersistenceids ● EventsByTag
  • 30. val readJournal = PersistenceQuery(system).readJournalFor(CassandraJournal.Identifier) val source = readJournal.query( EventsByPersistenceId(UserPersistenceId(name).persistenceId, 0, Long.MaxValue), NoRefresh) .map(_.event) .collect{ case s: EntireResistanceExerciseSession => s } .mapConcat(_.deviations) .filter(condition) .map(process) implicit val mat = ActorMaterializer() val result = source.runFold(List.empty[ExercisePlanDeviation])((x, y) => y :: x)
  • 31. ● Potentially infinite stream of events Source[Any].map(process).filter(condition) Publisher Subscriber process condition backpressure
  • 32. ● In Akka we have the read and write sides separated, in Cassandra we don’t ● Different data model ● Avoid using operational datastore ● Eventual consistency ● Streaming transformations to different format ● Unify journalled and other data
  • 33. ● Computations and analytics queries on the data ● Often iterative, complex, expensive computations ● Prepared and interactive queries ● Data from multiple sources, joins and transformations ● Often directly on a stream of data ● Whole history of events ● Historical behaviour ● Works retrospectively, can answer questions in the future that we don’t know exist yet ● Various data types from various sources ● Large amounts of fast data ● Automated analytics
  • 34. ● Cassandra 3.0 - user defined functions, functional indexes, aggregation functions, materialized views ● Server side denormalization ● Eventual consistency ● Copy of data with different partitioning userId performance
  • 35. ● In memory dataflow distributed data processing framework, streaming and batch ● Distributes computation using a higher level API ● Load balancing ● Moves computation to data ● Fault tolerant
  • 36. ● Resilient Distributed Datasets ● Fault tolerance ● Caching ● Serialization ● Transformations ○ Lazy, form the DAG ○ map, filter, flatMap, union, group, reduce, sort, join, repartition, cartesian, glom, ... ● Actions ○ Execute DAG, retrieve result ○ reduce, collect, count, first, take, foreach, saveAs…, min, max, ... ● Accumulators ● Broadcast Variables ● Integration ● Streaming ● Machine Learning ● Graph Processing
  • 37. textFile mapmap reduceByKey collect sc.textFile("counts") .map(line => line.split("t")) .map(word => (word(0), word(1).toInt)) .reduceByKey(_ + _) .collect() [4]
  • 39. ● Cassandra can store ● Spark can process ● Gathering large amounts of heterogeneous data ● Queries ● Transformations ● Complex computations ● Machine learning, data mining, analytics ● Now possible ● Prepared and interactive queries
  • 40. lazy val sparkConf: SparkConf = new SparkConf() .setAppName(...).setMaster(...).set("spark.cassandra.connection.host", "127.0.0.1") val sc = new SparkContext(sparkConf) val data = sc.cassandraTable[T]("keyspace", "table").select("columns") val processedData = data.flatMap(...)... processedData.saveToCassandra("keyspace", "table")
  • 41. ● Akka Analytics project ● Handles custom Akka serialization case class JournalKey(persistenceId: String, partition: Long, sequenceNr: Long) lazy val sparkConf: SparkConf = new SparkConf() .setAppName(...).setMaster(...).set("spark.cassandra.connection.host", "127.0.0.1") val sc = new SparkContext(sparkConf) val events: RDD[(JournalKey, Any)] = sc.eventTable() events.sortByKey().map(...).filter(...).collect().foreach(println)
  • 42. ● Spark streaming ● Precomputing using spark or replication often aiming for different data model Operational cluster Analytics cluster Precomputation / replication Integration with other data sources
  • 43. val events: RDD[(JournalKey, Any)] = sc.eventTable().cache().filterClass[EntireResistanceExerciseSession].flatMap(_.deviations) val deviationsFrequency = sqlContext.sql( """SELECT planned.exercise, hour(time), COUNT(1) FROM exerciseDeviations WHERE planned.exercise = 'bench press' GROUP BY planned.exercise, hour(time)""") val deviationsFrequency2 = exerciseDeviationsDF .where(exerciseDeviationsDF("planned.exercise") === "bench press") .groupBy( exerciseDeviationsDF("planned.exercise"), exerciseDeviationsDF("time”)) .count() val deviationsFrequency3 = exerciseDeviations .filter(_.planned.exercise == "bench press") .groupBy(d => (d.planned.exercise, d.time.getHours)) .map(d => (d._1, d._2.size))
  • 44. def toVector(user: User): mllib.linalg.Vector = Vectors.dense( user.frequency, user.performanceIndex, user.improvementIndex) val events: RDD[(JournalKey, Any)] = sc.eventTable().cache() val users: RDD[User] = events.filterClass[User] val kmeans = new KMeans() .setK(5) .set... val clusters = kmeans.run(users.map(_.toVector))
  • 45. val weight: RDD[(JournalKey, Any)] = sc.eventTable().cache() val exerciseDeviations = events .filterClass[EntireResistanceExerciseSession] .flatMap(session => session.sets.flatMap(set => set.sets.map(exercise => (session.id.id, exercise.exercise)))) .groupBy(e => e) .map(g => Rating(normalize(g._1._1), normalize(g._1._2), normalize(g._2.size))) val model = new ALS().run(ratings) val predictions = model.predict(recommend) bench press bicep curl dead lift user 1 5 2 user 2 4 3 user 3 5 2 user 4 3 1
  • 46. val events = sc.eventTable().cache().toDF() val lr = new LinearRegression() val pipeline = new Pipeline().setStages(Array(new UserFilter(), new ZScoreNormalizer(), new IntensityFeatureExtractor(), lr)) val paramGrid = new ParamGridBuilder() .addGrid(lr.regParam, Array(0.1, 0.01)) .addGrid(lr.fitIntercept, Array(true, false)) getEligibleUsers(events, sessionEndedBefore) .map { user => val trainValidationSplit = new TrainValidationSplit() .setEstimator(pipeline) .setEvaluator(new RegressionEvaluator) .setEstimatorParamMaps(paramGrid) val model = trainValidationSplit.fit( events, ParamMap(ParamPair(userIdParam, user))) val testData = // Prepare test data. val predictions = model.transform(testData) submitResult(userId, predictions, config) }
  • 47. val events: RDD[(JournalKey, Any)] = sc.eventTable().cache() val connections = events.filterClass[Connections] val vertices: RDD[(VertexId, Long)] = connections.map(c => (c.id, 1l)) val edges: RDD[Edge[Long]] = connections .flatMap(c => c.connections .map(Edge(c.id, _, 1l))) val graph = Graph(vertices, edges) val ranks = graph.pageRank(0.0001).vertices
  • 48.
  • 49.
  • 50.
  • 53. ● Exercise domain as an example ● Analytics of both batch (offline) and streaming (online) data ● Analytics important in other areas (banking, stock market, network, cluster monitoring, business intelligence, commerce, internet of things, ...) ● Enabling value of data
  • 54. ● Event sourcing ● CQRS ● Technologies to handle the data ○ Spark ○ Mesos ○ Akka ○ Cassandra ○ Kafka ● Handling data ● Insights and analytics enable value in data
  • 55.
  • 56. ● Jobs at www.cakesolutions.net/careers ● Code at https://github.com/muvr ● Martin Zapletal @zapletal_martin ● Anirvan Chakraborty @anirvan_c
  • 57. [1] http://www.benstopford.com/2015/04/28/elements-of-scale-composing-and-scaling-data-platforms/ [2] http://malteschwarzkopf.de/research/assets/google-stack.pdf [3] http://malteschwarzkopf.de/research/assets/facebook-stack.pdf [4] http://www.slideshare.net/LisaHua/spark-overview-37479609