Spark streaming
Exactly once
At least once + idempotence
Quentin Ambard
@qambard
© 2017 DataStax, All Rights Reserved.
Agenda
• Native checkpointing
• Driver & executor failure
• Custom checkpointing
• Performance tips
• Structured streaming (spark 2.2)
• Logs
• Kafka
• Bonus: Kafka 1 & “exactly-once”
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .2
Spark & kafka for (near)realtime
Once up a time, Spark met Kafka and…
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .4
P1
Kafka broker
…
Spark executor
Spark partition
…
Spark driver
Schedules jobs every X secs
Save the sum of
incoming
transaction
Start next job
Checkpoints
new offsets
for each
partition:
P1=5,
P2=4…
3
5
1
3
P2 P3
kafka partitions
offset
Spark native checkpointing
Native checkpointing
Spark saves for you the job state to a resilient storage (dsefs) :
• Save metadata, serialize the streaming context with:
• Spark configuration
• DStream operations (what your job is doing)
• Incomplete batches (scheduled but not yet executed)
• Last kafka offsets
Checkpoints twice: before and after each micro-batch (blocking)
Enabled with ssc.checkpoint(dir)
• Save the actual RDD data.
Should be enabled for stateful streams only (sliding window etc)
Expensive, enabled with kafkaStream.checkpoint(Seconds(10))
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .6
Checkpointing: example
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .7
val dir = "dsefs://35.188.108.191:5598/exactlyonce“
def createStreamingCtx() = {
val ssc = new StreamingContext(conf, Seconds(5))
val stream = KafkaUtils.createDirectStream(ssc, kafkaParams, topics)
//Apply transformations on each microbatch
stream.foreachRDD(rdd => println(rdd.count()))
//Set checkpointing directory
ssc.checkpoint(dir)
ssc
}
val ssc = StreamingContext.getOrCreate(dir, createStreamingCtx)
Checkpointing: restart behavior
• Read data saved in the checkpointing directory
• Schedule all missing job:
1 hour outage with a 1 seconds window will generate 60*60 micro-batches (!).
• First micro-batch starts from the last saved offsets
• First micro-batch will read all available kafka messages
Can be limited with
• spark.streaming.kafka.maxRatePerPartition = XXX
• spark.streaming.backpressure.enabled = true
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .8
Checkpointing: serialization
Serialize Dstream to disk
• Need to be serializable
• Won’t be able to restart a job if the Dstream operations have changed (classes are
differents)
This problem alone discard native checkpointing for a lot of customers
Workaround:
• Run the job twice in parallel during upgrade (can be an issue if the order
matters)
• “Clean stop”, restart from the beginning of the kafka topic© D a t a S t a x , A l l
R i g h t s R e s e r v e d .9
Checkpointing: speed
Executed twice per microbatch!
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .10
0
50
100
150
200
250
100 150 200 250 300 350 400 450 500 550 600 650 ou
plus...
frequency
latency (ms)
Basic job, saved to DSEFS (7.7 kb), 6 nodes
Failure handling
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .11
Worker failure
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .12
Scenario: the worker running the driver get killed
Master
Worker
Executor
Executor
Driver
(cluster-mode)
Server 1
Worker
Executor
Executor
Driver
(cluster-mode)
Server n
• 2 spark apps running at the same
time for a few sec
• Can break order
• Same checkpointing dir
Worker
• Master will restart the driver on
another worker
JVM
JVM
JVM
Driver
(cluster-mode)
JVM
• Driver will stop… Sometime…
(watches its worker)
Worker failure
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .13
Scenario: worker running the driver get killed
Concurrent operation on dsefs?
Worker failure
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .14
Workaround:
Make sure it’s the only instance running before starting the
stream
curl http://35.188.90.197:7080/api/v1/applications
or cql secret api ?
Checkpointing: failure in the driver
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .15
kafkaStream.foreachRDD(rdd => {
i += 1
println(s"Processing RDD $i")
if (i == 3) {
throw new RuntimeException(s"Exception for RDD $i")
}
val count = rdd.map(_._2+ "computed ").count()
println(s"RDD $i completed : $count")
})
Checkpointing: failure in the driver
Data from Job #2 (offset 100->200) is lost and won’t be recovered (erased by the next checkpointing)
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .16
Build RDD1
DAG
Job1
Offset 0->100
Executor 1
Executor n
Save medata
to dsefs
Build RDD1
DAG
Job2
Offset 100->200
Driver scheduler
Runtime exception
The main thread exits but
scheduler continue its job
Job execution continues with the
following microbatch
Build RDD1
DAG
Job3
Offset 200->300
Executor 1
Executor n
Save medata
to dsefs
Driver JVM
T secs T secs T secs
Checkpointing: failure in an executor
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .17
kafkaStream.foreachRDD(rdd => {
i += 1
println(s"Processing RDD $i")
val count = rdd.map(row => {
row._2+ "computed "
if (i == 3) {
throw new RuntimeException(s"Exception for RDD $i")
}}).count()
println(s"RDD $i completed : $count")
})
Inside
an executor
Checkpointing: failure in the driver
Data from Job #2 (offset 100->200) is lost and won’t be recovered (erased by the next checkpointing)
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .18
Build RDD1
DAG
Job1
Offset 0->100
Executor 1
Executor n
Save medata
to dsefs
Build RDD1
DAG
Job2
Offset 100->200
Driver scheduler
Retry spark.task.maxFailures,
throws exception in main thread
and continues
Build RDD1
DAG
Job3
Offset 200->300
Executor 1
Executor n
Save medata
to dsefs
Driver JVM
T secs T secs T secs
Executor 1
Executor n
Executor 1Executor 1
Checkpointing: fail-fast
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .19
Need a solution to stop job execution as soon as a
micro-batch fail
https://issues.apache.org/jira/browse/SPARK-6415
Workaround:
• Catch the error and kill the driver asap ?
Checkpointing: fail-fast
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .20
ssc.start()
Try {
ssc.awaitTermination()
} match {
case Success(result) =>
println("StreamingContext has been stopped")
case Failure(exception) =>
println("Stopping context and killing driver...", exception)
ssc.stop(stopSparkContext = true, stopGracefully = false)
System.exit(50) //sparkExitCode.UNCAUGHT_EXCEPTION
}
Checkpointing: fail-fast
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .21
• Streaming context waits 2 seconds before stopping (with
graceful = false).
• Scheduler continues its job and launch the next batch
• Shutdown hook to stop context on System.exit()
• Managed to reproduce with a Runtime.getRuntime.halt()
• Not guarantee that the next microbatch won’t be
checkpointed
Conclusion
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .22
Checkpointing: conclusion
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .23
Code/spark upgrade non-trivial (serialization issue)
Slow
Lot of corner-cases
Couldn’t find a way to make it 100% reliable
Reschedule all missed jobs during startup
Will likely kill your driver if it was stopped for a few days :/
Easy to setup (yeee)
Can’t easily monitor Kafka offset consumption
Custom checkpointing
Read kafka offset from rdd
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .25
val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges
offsetRanges.foreach{range =>
println(s"OFFSET RANGES for partition :${range.partition} =
[${range.fromOffset}-${range.untilOffset}]")
}
//OFFSET RANGES for partition :1 = [0-10]
//OFFSET RANGES for partition :2 = [0-13]
…
Stored with RDD metadata
Store kafka offset
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .26
def saveOffsetRanges (topic, ranges) = {
val rangesJson = ranges.sortBy(_.partition).toJson()
session.execute("insert into ks.checkpoint (topic, offsets) values (
${topic}, ${rangesJson} ")
}
Store data in C* yourself instead of hdfs
Doesn’t handle failures better than native checkpointing
Need to make sure the next batch won’t checkpoint until the previous is actually saved
Store kafka offset
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .27
def saveOffsetRanges (topic, ranges, previousRanges) = {
val rangesJson = ranges.sortBy(_.partition).toJson()
val rs = session.execute(" update ks.checkpoint set offsets=
${rangesJson} where topic=${topic} IF offsets=${previousRanges} ")
if (!rs.wasApplied){
log.error("last offsets doesn’t match. Killing driver. ")
System.exit(50)
}
rangesJson
}
Use a LWT to guarantee exactly-once
Ensure application uniqueness
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .28
//generated and updated to ks.checkpoint at application startup
val appUuid = UUIDs.random()
session.execute("update ks.checkpoint set app_id= ${appUuid} where
topic=${topic}")
….
def saveOffsetRanges (topic, ranges, previousRanges) = {
val rangesJson = ranges.sortBy(_.partition).toJson()
val rs = session.execute(“update ks.checkpoint set offsets=
${rangesJson} where topic=${topic} IF offsets=${previousRanges} and
app_id=${appUuid} ")
if (!rs.wasApplied){
…
}
When should we save offsets ?
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .29
kafkaStream.transform(rdd => {
rdd.doSomething…
}).foreachRDD(rdd => {
rdd.doSomething(…)
val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges
previousRanges = saveOffsetRanges(offsetRanges)
})
Always after the executor work
If you need to save the state at the beginning
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .30
kafkaStream.transform(rdd => {
//executed by job scheduler (LWT will fail with the first lag)
saveInitialState(rdd, topic, previousRange)
}).foreachRDD(rdd => {
rdd.doSomething…
previousRanges = saveOffsetRanges (rdd, topic, previousRange)
})
If you need to save the state at the beginning
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .31
kafkaStream.foreachRDD(rdd => {
//Must be protected by LWT
saveInitialState(rdd, topic, previousRange)
rdd.doSomething…
previousRanges = saveOffsetRanges (rdd, topic, previousRange)
})
Custom checkpointing: conclusion
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .32
Reliable
Harder to setup (Need to use LWT)
Kafka offset consumption easy to monitor
Fast
Easy upgrade
Performances tips
Performances
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .34
Very easy to get dead-slow throughput with complex jobs
(15 messages/sec for a solid 3 nodes cluster!)
The usual recommendations apply:
• Data distribution
• Avoid shuffle
• Prefer DataSet
• Use reducer instead of groupBy when possible…
Many kafka partitions with small process time?
• rdd.coalesce(numberOfExecutor)
• Disable spark.locality.wait
Performances
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .35
• Doing a lot of .cache() with multiple actions will hurt
• It’s often easier and faster to parallelize synchronous
computation in your executors
RDD[String]
RDD[Try[Transaction]]
Kafka raw message
RDD[Try[Transaction]]RDD[Try[Transaction]]
RDD[Error]
Map json to Scala class
Filter on errorFilter on success
RDD[Transaction]
continue job...
.cache()
Save to Cassandra
RDD[String]
Kafka raw message
Start multiple threads
and process each lines
sequentially, like any
non-distributed batch
http://www.russellspitzer.com/2017/02/27/Concurrency-In-Spark/
200ms: fast for long
batch, slow for a 3
seconds micro-batch
Performances
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .36
Monitor the driver and executor JVM !!
Typical scenario on driver:
• microbatches get slow for some reason
• Scheduler schedules many batches, add pressure on the heap
• Many GC, even slower, more lag, more GC, more lag, more GC…
On executors:
• By default, heap at 1GB
• “Only” (1024MB-300MB)*0.25 = 180MB for your objects!
(the rest is for spark storage and execution)
Structured streaming (2.2)
Unified API
• Uses DataSet instead of RDD
• Better support for aggregations
• Based on message timestamp
• Support “late data” (watermark)
• Stop/start queries
• …
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .38
Example
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .39
val ds = sparkSession.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "localhost:9092")
.option("subscribe", “myTopic")
.load()
.select(from_json($"value", schema).as("json"))
.select("json.*").as[Transaction]
val query = ds.writeStream
.cassandraFormat(“transaction“, ks)
.outputMode(OutputMode.Update)
.start()
How checkpointing works
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .40
• Checkpoint data to resilient storage (DSEFS, HDFS etc.)
• Once after each microbatch
• Conceptually similar to custom checkpointing
• Data saved as JSON
{"batchWatermarkMs":0,"batchTimestampMs":1502995346109,"conf":{"spark.sql.shuffl
e.partitions":"200"}}
{"exactlyonce":{"0":32625}}
Failures
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .41
• Fails fast and safely!
• Catch exception and safely restart the query without killing
the driver
• Still need to ensure the same query isn’t running twice
Checkpointing speed
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .42
DSEFS operations per microbatch: 9 read, 2 creation, 2 rename, 2 deletes
0
20
40
60
80
100
120
frequency
latency (ms)
Checkpointing latency
spark 2.2 -> DSEFS 5.1.2
Speed & Triggers
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .43
• Equivalent to window size
• 0 by default (= proceed asap)
• Increase throughput with a higher maxOffsetsPerTrigger
timeToProcess(maxOffsetsPerTrigger) >> checkpointing time
(otherwise spend too much time checkpointing)
ds.writeStream
.trigger(Trigger.ProcessingTime(“1 seconds"))
.queryName("exactlyOnce").foreach(writer).start
Faster checkpointing?
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .44
Implement a custom FileSystem to store on C* only (instead of DSEFS)
https://github.com/QuentinAmbard/cassandracheckointingfs/
class CassandraSimpleFileSystem extends hadoop.fs.FileSystem {
override def mkdirs(f: Path, permission: FsPermission): Boolean = ???
…
SparkSession.builder.config("spark.hadoop.fs.ckfs.impl",
"exactlyonce.CassandraSimpleFileSystem")
ds.writeStream.option("checkpointLocation",
"ckfs://127.0.0.1/checkpointing/exactlyOnce")
.queryName("exactlyOnce").foreach(writer).start
Custom checkpointing
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .45
0
100
200
300
400
500
600
frequency
latency (ms)
Checkpointing latency
spark 2.2 -> C* custom FS
Structured streaming: conclusion
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .46
Reliable
Fast
DataSet & powerful time-based aggregation (support for late data)
Easy upgrade (as long as aggregation type doesn’t change)
Currently slow with DSEFS
Old (blocked) data could be discarded if using producer time aggregation.
Use custom sink to reproduce previous streaming behavior
It’s the future!
Logs
Logs
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .48
spark-env.sh
Cleanup non-running application only
Won’t help for long-living streaming job
export SPARK_WORKER_OPTS="$SPARK_WORKER_OPTS -
Dspark.worker.cleanup.enabled=true -
Dspark.worker.cleanup.interval=300 -
Dspark.worker.cleanup.appDataTtl=300"
Logs
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .49
Limit the executor log size:
(SPARK_WORKER_DIR/worker-n/application_id/executor_id/…)
spark-defaults.conf
spark.executor.logs.rolling.maxRetainedFiles 3
spark.executor.logs.rolling.strategy size
spark.executor.logs.rolling.maxSize 50000
Logs
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .50
• If an executor jvm crashes, a new executor is started.
Creates new folder with a copy of the application .jar
• Crashed executors folder won’t be cleanup.
• Good practice to monitor the size of spark worker
directory
(but your executors shouldn’t OOM too often right?)
Kafka
Kafka durability
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .52
• Disable unclean leader election (now default)
• Define in-synch / RF (ex: RF=3, insynch=2)
• Publish at ALL (ack from all insynch)
• If order matters, allow only 1 inflight request (or 0 retries)
unclean.leader.election.enable = false
default.replication.factor = 3
min.insync.replicas=2
#producer
max.in.flight.requests.per.connection = 1
retries=3
acks = all
#consumer (disable autocommit for direct stream)
enable.auto.commit = false
How many kafka partitions ?
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .53
• Lot of partition will slow things down
• ~3x your number of spark cores is a good start
• Must allow to scale if needed
• If order matters, changing changing kafka partitioning
can be hard
Kafka 11: idempotent producer
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .54
• Idempotent producer:
• Generate a unique id per connection/producer
Your executors might restart with different producers id!
• Retry an unlimited number of time
• Broker de-duplicate based on the message id
Kafka 11: Transactions
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .55
props.put("transactional.id", “myId")
…
producer.initTransactions()
producer.beginTransaction()
kafka.producer.send(myProducerRecord1)
kafka.producer.send(myProducerRecord2)
kafka.producer.commitTransaction()
//Consumer configuration
props.put("isolation.level", "read_committed")
Kafka 11: Transactions
© D a t a S t a x , A l l
R i g h t s R e s e r v e d .56
• Consumer blocked to the last stable offset (LSO)
• 1 transaction stalled => LSO blocked => consumer
blocked
• Transaction.max.timeout.ms set to 900000 (15min) by
default
• “Exactly-once streams” based on transactions
Thank you
© 2017 DataStax, All Rights Reserved. Company Confidential

More Related Content

PPTX
Apache Tez - A New Chapter in Hadoop Data Processing
PPTX
Kafka replication apachecon_2013
PPSX
Event Sourcing & CQRS, Kafka, Rabbit MQ
PPSX
Microservices Docker Kubernetes Istio Kanban DevOps SRE
PDF
Exploring the power of OpenTelemetry on Kubernetes
PDF
Apache Kafka - Martin Podval
PPTX
Microservices Part 3 Service Mesh and Kafka
PDF
A Deep Dive into Query Execution Engine of Spark SQL
Apache Tez - A New Chapter in Hadoop Data Processing
Kafka replication apachecon_2013
Event Sourcing & CQRS, Kafka, Rabbit MQ
Microservices Docker Kubernetes Istio Kanban DevOps SRE
Exploring the power of OpenTelemetry on Kubernetes
Apache Kafka - Martin Podval
Microservices Part 3 Service Mesh and Kafka
A Deep Dive into Query Execution Engine of Spark SQL

What's hot (20)

PDF
Apache ZooKeeper
PPTX
Grokking Techtalk #37: Data intensive problem
PDF
Git training v10
PPTX
Kafka Summit NYC 2017 - Data Processing at LinkedIn with Apache Kafka
PPTX
Introduction to Apache Camel
PPTX
Getting up to speed with MirrorMaker 2 | Mickael Maison, IBM and Ryanne Dolan...
PDF
Introduction to Spring webflux
PPTX
Apache Kafka
PDF
Introducing Change Data Capture with Debezium
PPTX
A visual introduction to Apache Kafka
PDF
nexus helm 설치, docker/helm repo 설정과 예제
PDF
My first 90 days with ClickHouse.pdf
PPTX
Netflix Data Pipeline With Kafka
PPTX
Apache Tez – Present and Future
PDF
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
PDF
Scylla Summit 2022: The Future of Consensus in ScyllaDB 5.0 and Beyond
PPTX
Introduction to Apache Kafka
PPTX
Apache kafka
PDF
PDF
Paris Redis Meetup Introduction
Apache ZooKeeper
Grokking Techtalk #37: Data intensive problem
Git training v10
Kafka Summit NYC 2017 - Data Processing at LinkedIn with Apache Kafka
Introduction to Apache Camel
Getting up to speed with MirrorMaker 2 | Mickael Maison, IBM and Ryanne Dolan...
Introduction to Spring webflux
Apache Kafka
Introducing Change Data Capture with Debezium
A visual introduction to Apache Kafka
nexus helm 설치, docker/helm repo 설정과 예제
My first 90 days with ClickHouse.pdf
Netflix Data Pipeline With Kafka
Apache Tez – Present and Future
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Scylla Summit 2022: The Future of Consensus in ScyllaDB 5.0 and Beyond
Introduction to Apache Kafka
Apache kafka
Paris Redis Meetup Introduction
Ad

Similar to Exactly once with spark streaming (20)

PDF
Deathstar
PDF
Beyond the Query – Bringing Complex Access Patterns to NoSQL with DataStax - ...
PDF
Beyond the Query: A Cassandra + Solr + Spark Love Triangle Using Datastax Ent...
PPTX
Tuning and Debugging in Apache Spark
PPTX
Apache Spark Workshop
PPTX
Advanced Replication
PDF
Osol Pgsql
PPTX
Big Data Scala by the Bay: Interactive Spark in your Browser
PDF
A Cassandra + Solr + Spark Love Triangle Using DataStax Enterprise
PPTX
Intro to Spark - for Denver Big Data Meetup
PDF
Tuning and Debugging in Apache Spark
KEY
Introduction to cloudforecast
PDF
Performance tweaks and tools for Linux (Joe Damato)
PDF
Spark Streaming Programming Techniques You Should Know with Gerard Maas
PDF
Redis acc
PPTX
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
PDF
Debugging Ruby Systems
PPT
11. From Hadoop to Spark 2/2
PDF
D Trace Support In My Sql Guide To Solving Reallife Performance Problems
PDF
2013 london advanced-replication
Deathstar
Beyond the Query – Bringing Complex Access Patterns to NoSQL with DataStax - ...
Beyond the Query: A Cassandra + Solr + Spark Love Triangle Using Datastax Ent...
Tuning and Debugging in Apache Spark
Apache Spark Workshop
Advanced Replication
Osol Pgsql
Big Data Scala by the Bay: Interactive Spark in your Browser
A Cassandra + Solr + Spark Love Triangle Using DataStax Enterprise
Intro to Spark - for Denver Big Data Meetup
Tuning and Debugging in Apache Spark
Introduction to cloudforecast
Performance tweaks and tools for Linux (Joe Damato)
Spark Streaming Programming Techniques You Should Know with Gerard Maas
Redis acc
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Debugging Ruby Systems
11. From Hadoop to Spark 2/2
D Trace Support In My Sql Guide To Solving Reallife Performance Problems
2013 london advanced-replication
Ad

Recently uploaded (20)

PDF
Five Habits of High-Impact Board Members
PPTX
Chapter 5: Probability Theory and Statistics
PDF
Two-dimensional Klein-Gordon and Sine-Gordon numerical solutions based on dee...
PDF
A review of recent deep learning applications in wood surface defect identifi...
PDF
ENT215_Completing-a-large-scale-migration-and-modernization-with-AWS.pdf
PDF
A Late Bloomer's Guide to GenAI: Ethics, Bias, and Effective Prompting - Boha...
PDF
A proposed approach for plagiarism detection in Myanmar Unicode text
PDF
Flame analysis and combustion estimation using large language and vision assi...
PPTX
2018-HIPAA-Renewal-Training for executives
PPTX
Modernising the Digital Integration Hub
PDF
Taming the Chaos: How to Turn Unstructured Data into Decisions
PDF
Produktkatalog für HOBO Datenlogger, Wetterstationen, Sensoren, Software und ...
PPT
What is a Computer? Input Devices /output devices
PDF
NewMind AI Weekly Chronicles – August ’25 Week III
PPTX
Benefits of Physical activity for teenagers.pptx
PDF
UiPath Agentic Automation session 1: RPA to Agents
DOCX
search engine optimization ppt fir known well about this
PDF
sbt 2.0: go big (Scala Days 2025 edition)
PDF
Abstractive summarization using multilingual text-to-text transfer transforme...
PPTX
MicrosoftCybserSecurityReferenceArchitecture-April-2025.pptx
Five Habits of High-Impact Board Members
Chapter 5: Probability Theory and Statistics
Two-dimensional Klein-Gordon and Sine-Gordon numerical solutions based on dee...
A review of recent deep learning applications in wood surface defect identifi...
ENT215_Completing-a-large-scale-migration-and-modernization-with-AWS.pdf
A Late Bloomer's Guide to GenAI: Ethics, Bias, and Effective Prompting - Boha...
A proposed approach for plagiarism detection in Myanmar Unicode text
Flame analysis and combustion estimation using large language and vision assi...
2018-HIPAA-Renewal-Training for executives
Modernising the Digital Integration Hub
Taming the Chaos: How to Turn Unstructured Data into Decisions
Produktkatalog für HOBO Datenlogger, Wetterstationen, Sensoren, Software und ...
What is a Computer? Input Devices /output devices
NewMind AI Weekly Chronicles – August ’25 Week III
Benefits of Physical activity for teenagers.pptx
UiPath Agentic Automation session 1: RPA to Agents
search engine optimization ppt fir known well about this
sbt 2.0: go big (Scala Days 2025 edition)
Abstractive summarization using multilingual text-to-text transfer transforme...
MicrosoftCybserSecurityReferenceArchitecture-April-2025.pptx

Exactly once with spark streaming

  • 1. Spark streaming Exactly once At least once + idempotence Quentin Ambard @qambard © 2017 DataStax, All Rights Reserved.
  • 2. Agenda • Native checkpointing • Driver & executor failure • Custom checkpointing • Performance tips • Structured streaming (spark 2.2) • Logs • Kafka • Bonus: Kafka 1 & “exactly-once” © D a t a S t a x , A l l R i g h t s R e s e r v e d .2
  • 3. Spark & kafka for (near)realtime
  • 4. Once up a time, Spark met Kafka and… © D a t a S t a x , A l l R i g h t s R e s e r v e d .4 P1 Kafka broker … Spark executor Spark partition … Spark driver Schedules jobs every X secs Save the sum of incoming transaction Start next job Checkpoints new offsets for each partition: P1=5, P2=4… 3 5 1 3 P2 P3 kafka partitions offset
  • 6. Native checkpointing Spark saves for you the job state to a resilient storage (dsefs) : • Save metadata, serialize the streaming context with: • Spark configuration • DStream operations (what your job is doing) • Incomplete batches (scheduled but not yet executed) • Last kafka offsets Checkpoints twice: before and after each micro-batch (blocking) Enabled with ssc.checkpoint(dir) • Save the actual RDD data. Should be enabled for stateful streams only (sliding window etc) Expensive, enabled with kafkaStream.checkpoint(Seconds(10)) © D a t a S t a x , A l l R i g h t s R e s e r v e d .6
  • 7. Checkpointing: example © D a t a S t a x , A l l R i g h t s R e s e r v e d .7 val dir = "dsefs://35.188.108.191:5598/exactlyonce“ def createStreamingCtx() = { val ssc = new StreamingContext(conf, Seconds(5)) val stream = KafkaUtils.createDirectStream(ssc, kafkaParams, topics) //Apply transformations on each microbatch stream.foreachRDD(rdd => println(rdd.count())) //Set checkpointing directory ssc.checkpoint(dir) ssc } val ssc = StreamingContext.getOrCreate(dir, createStreamingCtx)
  • 8. Checkpointing: restart behavior • Read data saved in the checkpointing directory • Schedule all missing job: 1 hour outage with a 1 seconds window will generate 60*60 micro-batches (!). • First micro-batch starts from the last saved offsets • First micro-batch will read all available kafka messages Can be limited with • spark.streaming.kafka.maxRatePerPartition = XXX • spark.streaming.backpressure.enabled = true © D a t a S t a x , A l l R i g h t s R e s e r v e d .8
  • 9. Checkpointing: serialization Serialize Dstream to disk • Need to be serializable • Won’t be able to restart a job if the Dstream operations have changed (classes are differents) This problem alone discard native checkpointing for a lot of customers Workaround: • Run the job twice in parallel during upgrade (can be an issue if the order matters) • “Clean stop”, restart from the beginning of the kafka topic© D a t a S t a x , A l l R i g h t s R e s e r v e d .9
  • 10. Checkpointing: speed Executed twice per microbatch! © D a t a S t a x , A l l R i g h t s R e s e r v e d .10 0 50 100 150 200 250 100 150 200 250 300 350 400 450 500 550 600 650 ou plus... frequency latency (ms) Basic job, saved to DSEFS (7.7 kb), 6 nodes
  • 11. Failure handling © D a t a S t a x , A l l R i g h t s R e s e r v e d .11
  • 12. Worker failure © D a t a S t a x , A l l R i g h t s R e s e r v e d .12 Scenario: the worker running the driver get killed Master Worker Executor Executor Driver (cluster-mode) Server 1 Worker Executor Executor Driver (cluster-mode) Server n • 2 spark apps running at the same time for a few sec • Can break order • Same checkpointing dir Worker • Master will restart the driver on another worker JVM JVM JVM Driver (cluster-mode) JVM • Driver will stop… Sometime… (watches its worker)
  • 13. Worker failure © D a t a S t a x , A l l R i g h t s R e s e r v e d .13 Scenario: worker running the driver get killed Concurrent operation on dsefs?
  • 14. Worker failure © D a t a S t a x , A l l R i g h t s R e s e r v e d .14 Workaround: Make sure it’s the only instance running before starting the stream curl http://35.188.90.197:7080/api/v1/applications or cql secret api ?
  • 15. Checkpointing: failure in the driver © D a t a S t a x , A l l R i g h t s R e s e r v e d .15 kafkaStream.foreachRDD(rdd => { i += 1 println(s"Processing RDD $i") if (i == 3) { throw new RuntimeException(s"Exception for RDD $i") } val count = rdd.map(_._2+ "computed ").count() println(s"RDD $i completed : $count") })
  • 16. Checkpointing: failure in the driver Data from Job #2 (offset 100->200) is lost and won’t be recovered (erased by the next checkpointing) © D a t a S t a x , A l l R i g h t s R e s e r v e d .16 Build RDD1 DAG Job1 Offset 0->100 Executor 1 Executor n Save medata to dsefs Build RDD1 DAG Job2 Offset 100->200 Driver scheduler Runtime exception The main thread exits but scheduler continue its job Job execution continues with the following microbatch Build RDD1 DAG Job3 Offset 200->300 Executor 1 Executor n Save medata to dsefs Driver JVM T secs T secs T secs
  • 17. Checkpointing: failure in an executor © D a t a S t a x , A l l R i g h t s R e s e r v e d .17 kafkaStream.foreachRDD(rdd => { i += 1 println(s"Processing RDD $i") val count = rdd.map(row => { row._2+ "computed " if (i == 3) { throw new RuntimeException(s"Exception for RDD $i") }}).count() println(s"RDD $i completed : $count") }) Inside an executor
  • 18. Checkpointing: failure in the driver Data from Job #2 (offset 100->200) is lost and won’t be recovered (erased by the next checkpointing) © D a t a S t a x , A l l R i g h t s R e s e r v e d .18 Build RDD1 DAG Job1 Offset 0->100 Executor 1 Executor n Save medata to dsefs Build RDD1 DAG Job2 Offset 100->200 Driver scheduler Retry spark.task.maxFailures, throws exception in main thread and continues Build RDD1 DAG Job3 Offset 200->300 Executor 1 Executor n Save medata to dsefs Driver JVM T secs T secs T secs Executor 1 Executor n Executor 1Executor 1
  • 19. Checkpointing: fail-fast © D a t a S t a x , A l l R i g h t s R e s e r v e d .19 Need a solution to stop job execution as soon as a micro-batch fail https://issues.apache.org/jira/browse/SPARK-6415 Workaround: • Catch the error and kill the driver asap ?
  • 20. Checkpointing: fail-fast © D a t a S t a x , A l l R i g h t s R e s e r v e d .20 ssc.start() Try { ssc.awaitTermination() } match { case Success(result) => println("StreamingContext has been stopped") case Failure(exception) => println("Stopping context and killing driver...", exception) ssc.stop(stopSparkContext = true, stopGracefully = false) System.exit(50) //sparkExitCode.UNCAUGHT_EXCEPTION }
  • 21. Checkpointing: fail-fast © D a t a S t a x , A l l R i g h t s R e s e r v e d .21 • Streaming context waits 2 seconds before stopping (with graceful = false). • Scheduler continues its job and launch the next batch • Shutdown hook to stop context on System.exit() • Managed to reproduce with a Runtime.getRuntime.halt() • Not guarantee that the next microbatch won’t be checkpointed
  • 22. Conclusion © D a t a S t a x , A l l R i g h t s R e s e r v e d .22
  • 23. Checkpointing: conclusion © D a t a S t a x , A l l R i g h t s R e s e r v e d .23 Code/spark upgrade non-trivial (serialization issue) Slow Lot of corner-cases Couldn’t find a way to make it 100% reliable Reschedule all missed jobs during startup Will likely kill your driver if it was stopped for a few days :/ Easy to setup (yeee) Can’t easily monitor Kafka offset consumption
  • 25. Read kafka offset from rdd © D a t a S t a x , A l l R i g h t s R e s e r v e d .25 val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges offsetRanges.foreach{range => println(s"OFFSET RANGES for partition :${range.partition} = [${range.fromOffset}-${range.untilOffset}]") } //OFFSET RANGES for partition :1 = [0-10] //OFFSET RANGES for partition :2 = [0-13] … Stored with RDD metadata
  • 26. Store kafka offset © D a t a S t a x , A l l R i g h t s R e s e r v e d .26 def saveOffsetRanges (topic, ranges) = { val rangesJson = ranges.sortBy(_.partition).toJson() session.execute("insert into ks.checkpoint (topic, offsets) values ( ${topic}, ${rangesJson} ") } Store data in C* yourself instead of hdfs Doesn’t handle failures better than native checkpointing Need to make sure the next batch won’t checkpoint until the previous is actually saved
  • 27. Store kafka offset © D a t a S t a x , A l l R i g h t s R e s e r v e d .27 def saveOffsetRanges (topic, ranges, previousRanges) = { val rangesJson = ranges.sortBy(_.partition).toJson() val rs = session.execute(" update ks.checkpoint set offsets= ${rangesJson} where topic=${topic} IF offsets=${previousRanges} ") if (!rs.wasApplied){ log.error("last offsets doesn’t match. Killing driver. ") System.exit(50) } rangesJson } Use a LWT to guarantee exactly-once
  • 28. Ensure application uniqueness © D a t a S t a x , A l l R i g h t s R e s e r v e d .28 //generated and updated to ks.checkpoint at application startup val appUuid = UUIDs.random() session.execute("update ks.checkpoint set app_id= ${appUuid} where topic=${topic}") …. def saveOffsetRanges (topic, ranges, previousRanges) = { val rangesJson = ranges.sortBy(_.partition).toJson() val rs = session.execute(“update ks.checkpoint set offsets= ${rangesJson} where topic=${topic} IF offsets=${previousRanges} and app_id=${appUuid} ") if (!rs.wasApplied){ … }
  • 29. When should we save offsets ? © D a t a S t a x , A l l R i g h t s R e s e r v e d .29 kafkaStream.transform(rdd => { rdd.doSomething… }).foreachRDD(rdd => { rdd.doSomething(…) val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges previousRanges = saveOffsetRanges(offsetRanges) }) Always after the executor work
  • 30. If you need to save the state at the beginning © D a t a S t a x , A l l R i g h t s R e s e r v e d .30 kafkaStream.transform(rdd => { //executed by job scheduler (LWT will fail with the first lag) saveInitialState(rdd, topic, previousRange) }).foreachRDD(rdd => { rdd.doSomething… previousRanges = saveOffsetRanges (rdd, topic, previousRange) })
  • 31. If you need to save the state at the beginning © D a t a S t a x , A l l R i g h t s R e s e r v e d .31 kafkaStream.foreachRDD(rdd => { //Must be protected by LWT saveInitialState(rdd, topic, previousRange) rdd.doSomething… previousRanges = saveOffsetRanges (rdd, topic, previousRange) })
  • 32. Custom checkpointing: conclusion © D a t a S t a x , A l l R i g h t s R e s e r v e d .32 Reliable Harder to setup (Need to use LWT) Kafka offset consumption easy to monitor Fast Easy upgrade
  • 34. Performances © D a t a S t a x , A l l R i g h t s R e s e r v e d .34 Very easy to get dead-slow throughput with complex jobs (15 messages/sec for a solid 3 nodes cluster!) The usual recommendations apply: • Data distribution • Avoid shuffle • Prefer DataSet • Use reducer instead of groupBy when possible… Many kafka partitions with small process time? • rdd.coalesce(numberOfExecutor) • Disable spark.locality.wait
  • 35. Performances © D a t a S t a x , A l l R i g h t s R e s e r v e d .35 • Doing a lot of .cache() with multiple actions will hurt • It’s often easier and faster to parallelize synchronous computation in your executors RDD[String] RDD[Try[Transaction]] Kafka raw message RDD[Try[Transaction]]RDD[Try[Transaction]] RDD[Error] Map json to Scala class Filter on errorFilter on success RDD[Transaction] continue job... .cache() Save to Cassandra RDD[String] Kafka raw message Start multiple threads and process each lines sequentially, like any non-distributed batch http://www.russellspitzer.com/2017/02/27/Concurrency-In-Spark/ 200ms: fast for long batch, slow for a 3 seconds micro-batch
  • 36. Performances © D a t a S t a x , A l l R i g h t s R e s e r v e d .36 Monitor the driver and executor JVM !! Typical scenario on driver: • microbatches get slow for some reason • Scheduler schedules many batches, add pressure on the heap • Many GC, even slower, more lag, more GC, more lag, more GC… On executors: • By default, heap at 1GB • “Only” (1024MB-300MB)*0.25 = 180MB for your objects! (the rest is for spark storage and execution)
  • 38. Unified API • Uses DataSet instead of RDD • Better support for aggregations • Based on message timestamp • Support “late data” (watermark) • Stop/start queries • … © D a t a S t a x , A l l R i g h t s R e s e r v e d .38
  • 39. Example © D a t a S t a x , A l l R i g h t s R e s e r v e d .39 val ds = sparkSession.readStream .format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", “myTopic") .load() .select(from_json($"value", schema).as("json")) .select("json.*").as[Transaction] val query = ds.writeStream .cassandraFormat(“transaction“, ks) .outputMode(OutputMode.Update) .start()
  • 40. How checkpointing works © D a t a S t a x , A l l R i g h t s R e s e r v e d .40 • Checkpoint data to resilient storage (DSEFS, HDFS etc.) • Once after each microbatch • Conceptually similar to custom checkpointing • Data saved as JSON {"batchWatermarkMs":0,"batchTimestampMs":1502995346109,"conf":{"spark.sql.shuffl e.partitions":"200"}} {"exactlyonce":{"0":32625}}
  • 41. Failures © D a t a S t a x , A l l R i g h t s R e s e r v e d .41 • Fails fast and safely! • Catch exception and safely restart the query without killing the driver • Still need to ensure the same query isn’t running twice
  • 42. Checkpointing speed © D a t a S t a x , A l l R i g h t s R e s e r v e d .42 DSEFS operations per microbatch: 9 read, 2 creation, 2 rename, 2 deletes 0 20 40 60 80 100 120 frequency latency (ms) Checkpointing latency spark 2.2 -> DSEFS 5.1.2
  • 43. Speed & Triggers © D a t a S t a x , A l l R i g h t s R e s e r v e d .43 • Equivalent to window size • 0 by default (= proceed asap) • Increase throughput with a higher maxOffsetsPerTrigger timeToProcess(maxOffsetsPerTrigger) >> checkpointing time (otherwise spend too much time checkpointing) ds.writeStream .trigger(Trigger.ProcessingTime(“1 seconds")) .queryName("exactlyOnce").foreach(writer).start
  • 44. Faster checkpointing? © D a t a S t a x , A l l R i g h t s R e s e r v e d .44 Implement a custom FileSystem to store on C* only (instead of DSEFS) https://github.com/QuentinAmbard/cassandracheckointingfs/ class CassandraSimpleFileSystem extends hadoop.fs.FileSystem { override def mkdirs(f: Path, permission: FsPermission): Boolean = ??? … SparkSession.builder.config("spark.hadoop.fs.ckfs.impl", "exactlyonce.CassandraSimpleFileSystem") ds.writeStream.option("checkpointLocation", "ckfs://127.0.0.1/checkpointing/exactlyOnce") .queryName("exactlyOnce").foreach(writer).start
  • 45. Custom checkpointing © D a t a S t a x , A l l R i g h t s R e s e r v e d .45 0 100 200 300 400 500 600 frequency latency (ms) Checkpointing latency spark 2.2 -> C* custom FS
  • 46. Structured streaming: conclusion © D a t a S t a x , A l l R i g h t s R e s e r v e d .46 Reliable Fast DataSet & powerful time-based aggregation (support for late data) Easy upgrade (as long as aggregation type doesn’t change) Currently slow with DSEFS Old (blocked) data could be discarded if using producer time aggregation. Use custom sink to reproduce previous streaming behavior It’s the future!
  • 47. Logs
  • 48. Logs © D a t a S t a x , A l l R i g h t s R e s e r v e d .48 spark-env.sh Cleanup non-running application only Won’t help for long-living streaming job export SPARK_WORKER_OPTS="$SPARK_WORKER_OPTS - Dspark.worker.cleanup.enabled=true - Dspark.worker.cleanup.interval=300 - Dspark.worker.cleanup.appDataTtl=300"
  • 49. Logs © D a t a S t a x , A l l R i g h t s R e s e r v e d .49 Limit the executor log size: (SPARK_WORKER_DIR/worker-n/application_id/executor_id/…) spark-defaults.conf spark.executor.logs.rolling.maxRetainedFiles 3 spark.executor.logs.rolling.strategy size spark.executor.logs.rolling.maxSize 50000
  • 50. Logs © D a t a S t a x , A l l R i g h t s R e s e r v e d .50 • If an executor jvm crashes, a new executor is started. Creates new folder with a copy of the application .jar • Crashed executors folder won’t be cleanup. • Good practice to monitor the size of spark worker directory (but your executors shouldn’t OOM too often right?)
  • 51. Kafka
  • 52. Kafka durability © D a t a S t a x , A l l R i g h t s R e s e r v e d .52 • Disable unclean leader election (now default) • Define in-synch / RF (ex: RF=3, insynch=2) • Publish at ALL (ack from all insynch) • If order matters, allow only 1 inflight request (or 0 retries) unclean.leader.election.enable = false default.replication.factor = 3 min.insync.replicas=2 #producer max.in.flight.requests.per.connection = 1 retries=3 acks = all #consumer (disable autocommit for direct stream) enable.auto.commit = false
  • 53. How many kafka partitions ? © D a t a S t a x , A l l R i g h t s R e s e r v e d .53 • Lot of partition will slow things down • ~3x your number of spark cores is a good start • Must allow to scale if needed • If order matters, changing changing kafka partitioning can be hard
  • 54. Kafka 11: idempotent producer © D a t a S t a x , A l l R i g h t s R e s e r v e d .54 • Idempotent producer: • Generate a unique id per connection/producer Your executors might restart with different producers id! • Retry an unlimited number of time • Broker de-duplicate based on the message id
  • 55. Kafka 11: Transactions © D a t a S t a x , A l l R i g h t s R e s e r v e d .55 props.put("transactional.id", “myId") … producer.initTransactions() producer.beginTransaction() kafka.producer.send(myProducerRecord1) kafka.producer.send(myProducerRecord2) kafka.producer.commitTransaction() //Consumer configuration props.put("isolation.level", "read_committed")
  • 56. Kafka 11: Transactions © D a t a S t a x , A l l R i g h t s R e s e r v e d .56 • Consumer blocked to the last stable offset (LSO) • 1 transaction stalled => LSO blocked => consumer blocked • Transaction.max.timeout.ms set to 900000 (15min) by default • “Exactly-once streams” based on transactions
  • 57. Thank you © 2017 DataStax, All Rights Reserved. Company Confidential

Editor's Notes

  • #2: Hello and thanks for joining us in this discussion today
  • #4: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #5: see JobGenerator.scala
  • #6: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #7: see JobGenerator.scala
  • #25: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #34: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #38: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #40: Spark context can be killed with streaming context still running. Will enter the foreachRDD loop without having any executor available. saveInitialState(…) LWT protects against this. If needed could add a synchronization point before: rdd.take(1)
  • #48: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #52: So who is DataStax? We are a 400 people company with offices around the world. We have validation from some of the worlds leading venture firms And we are continuing to drive momentum with customers that are the top brands in the world who believe in us Even Industry influencers like Gartner consistently rank us at the top in multiple categories. Last year Forbes listed us in the world’s best 100 cloud companies.
  • #54: Disable unclean leader election Disable unclean leader election to prevent the election of a leader out of sync (could result in data loss). This provides balance between consistency (guaranteed message delivery) and availability. With unclean leader election disabled, if a broker containing the leader replica for a partition becomes unavailable, and no in-sync replica exists to replace it, the partition becomes unavailable until the leader replica or another in-sync replica is back online. unclean.leader.election.enable = false Replication factor RF=3 is probably the best choice for this use case default.replication.factor = 3 Max inflight request (order guarantee) If the order matters, set the max number of inflight request to 1 (otherwise the first one could fail and the 2nd one succeed) Max.in.flight.requests.per.connection = 1 Minimum ISRs Change the minimum in-synch replicas that must be available for the producer to successfully send messages to a partition. A replica is considered in synch as long as the lag is < Replica.lag.max.messages and the leader received the last fetch within replica.lag.time.max.ms In newer versions, replica.lag.time.max.ms also put a replica out of synch if he didn’t catch up to the last upset within this duration. Minimum ISR should be set to 2 to prevent data loss min.insynch.replicas = 2 Producer acks Producer should wait for all ISR ack (at least 2) to prevent message lost acks = all
  • #58: Thank you.