-
Notifications
You must be signed in to change notification settings - Fork 10
/
CassandraJournal.scala
931 lines (821 loc) · 37.8 KB
/
CassandraJournal.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* license agreements; and to You under the Apache License, version 2.0:
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* This file is part of the Apache Pekko project, which was derived from Akka.
*/
/*
* Copyright (C) 2016-2020 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence.cassandra.journal
import java.lang.{ Long => JLong }
import java.nio.ByteBuffer
import java.util.{ HashMap => JHMap, Map => JMap, UUID }
import org.apache.pekko
import pekko.Done
import pekko.actor.SupervisorStrategy.Stop
import pekko.actor._
import pekko.annotation.{ DoNotInherit, InternalApi, InternalStableApi }
import pekko.dispatch.ExecutionContexts
import pekko.event.{ Logging, LoggingAdapter }
import pekko.pattern.{ ask, pipe }
import pekko.persistence._
import pekko.persistence.cassandra._
import pekko.persistence.cassandra.Extractors
import pekko.persistence.cassandra.query.scaladsl.CassandraReadJournal
import pekko.persistence.journal.{ AsyncWriteJournal, Tagged }
import pekko.persistence.query.PersistenceQuery
import pekko.persistence.cassandra.journal.TagWriters.{ BulkTagWrite, TagWrite, TagWritersSession }
import pekko.persistence.cassandra.journal.TagWriter.TagProgress
import pekko.serialization.{ AsyncSerializer, Serialization, SerializationExtension }
import pekko.stream.connectors.cassandra.scaladsl.{ CassandraSession, CassandraSessionRegistry }
import pekko.stream.scaladsl.{ Sink, Source }
import pekko.util.{ OptionVal, Timeout }
import pekko.util.FutureConverters._
import pekko.util.ccompat.JavaConverters._
import com.datastax.oss.driver.api.core.cql._
import com.typesafe.config.Config
import com.datastax.oss.driver.api.core.uuid.Uuids
import com.datastax.oss.protocol.internal.util.Bytes
import scala.annotation.tailrec
import scala.collection.immutable
import scala.collection.immutable.Seq
import scala.concurrent._
import scala.util.control.NonFatal
import scala.util.{ Failure, Success, Try }
/**
* INTERNAL API
*
* Journal implementation of the cassandra plugin.
*/
@DoNotInherit
@InternalStableApi // extended by Multi-DC Persistence
@InternalApi private[pekko] class CassandraJournal(cfg: Config, cfgPath: String)
extends AsyncWriteJournal
with NoSerializationVerificationNeeded {
import CassandraJournal._
import context.system
// shared config is one level above the journal specific
private val sharedConfigPath = cfgPath.replaceAll("""\.journal$""", "")
private val sharedConfig = context.system.settings.config.getConfig(sharedConfigPath)
private val settings = new PluginSettings(context.system, sharedConfig)
import settings._
private val eventDeserializer: CassandraJournal.EventDeserializer =
new CassandraJournal.EventDeserializer(context.system)
private val statements: CassandraStatements = new CassandraStatements(settings)
private val healthCheckCql = settings.healthCheckSettings.healthCheckCql
private val serialization = SerializationExtension(context.system)
private val log: LoggingAdapter = Logging(context.system, classOf[CassandraJournal])
private implicit val ec: ExecutionContext = context.dispatcher
// readHighestSequence must be performed after pending write for a persistenceId
// when the persistent actor is restarted.
// It seems like C* doesn't support session consistency so we handle it ourselves.
// https://aphyr.com/posts/299-the-trouble-with-timestamps
private val writeInProgress: JMap[String, Future[Done]] = new JHMap
// Can't think of a reason why we can't have writes and deletes
// run concurrently. This should be a very infrequently used
// so fine to use an immutable list as the value
private val pendingDeletes: JMap[String, List[PendingDelete]] = new JHMap
private val session: CassandraSession =
CassandraSessionRegistry(context.system)
.sessionFor(sharedConfigPath, ses => statements.executeAllCreateKeyspaceAndTables(ses, log))
private val taggedPreparedStatements = new TaggedPreparedStatements(statements.journalStatements, session.prepare)
private val tagWriterSession = TagWritersSession(
session,
settings.journalSettings.writeProfile,
settings.journalSettings.readProfile,
taggedPreparedStatements)
private val tagWrites: Option[ActorRef] =
if (settings.eventsByTagSettings.eventsByTagEnabled)
Some(
context.actorOf(
TagWriters
.props(settings.eventsByTagSettings.tagWriterSettings, tagWriterSession)
.withDispatcher(context.props.dispatcher),
"tagWrites"))
else None
private val tagRecovery: Option[CassandraTagRecovery] =
tagWrites.map(ref => new CassandraTagRecovery(context.system, session, settings, taggedPreparedStatements, ref))
private lazy val preparedWriteMessage =
session.prepare(statements.journalStatements.writeMessage(withMeta = false))
private lazy val preparedSelectDeletedTo: Option[Future[PreparedStatement]] = {
if (settings.journalSettings.supportDeletes)
Some(session.prepare(statements.journalStatements.selectDeletedTo))
else
None
}
private lazy val preparedSelectHighestSequenceNr: Future[PreparedStatement] =
session.prepare(statements.journalStatements.selectHighestSequenceNr)
private def deletesNotSupportedException: Future[PreparedStatement] =
Future.failed(new IllegalArgumentException(s"Deletes not supported because config support-deletes=off"))
private lazy val preparedInsertDeletedTo: Future[PreparedStatement] = {
if (settings.journalSettings.supportDeletes)
session.prepare(statements.journalStatements.insertDeletedTo)
else
deletesNotSupportedException
}
private lazy val preparedDeleteMessages: Future[PreparedStatement] = {
if (settings.journalSettings.supportDeletes) {
session.serverMetaData.flatMap { meta =>
session.prepare(statements.journalStatements.deleteMessages(meta.isVersion2 || settings.cosmosDb))
}
} else
deletesNotSupportedException
}
private lazy val preparedInsertIntoAllPersistenceIds: Future[PreparedStatement] = {
session.prepare(statements.journalStatements.insertIntoAllPersistenceIds)
}
private lazy val preparedWriteMessageWithMeta =
session.prepare(statements.journalStatements.writeMessage(withMeta = true))
private lazy val preparedSelectMessages =
session.prepare(statements.journalStatements.selectMessages)
private lazy val queries: CassandraReadJournal =
PersistenceQuery(context.system.asInstanceOf[ExtendedActorSystem])
.readJournalFor[CassandraReadJournal](s"$sharedConfigPath.query")
// For TagWriters/TagWriter children
override def supervisorStrategy: SupervisorStrategy = OneForOneStrategy() {
case e: Exception =>
log.error(e, "Cassandra Journal has experienced an unexpected error and requires an ActorSystem restart.")
if (settings.journalSettings.coordinatedShutdownOnError) {
CoordinatedShutdown(context.system).run(CassandraJournalUnexpectedError)
}
context.stop(context.self)
Stop
}
override def preStart(): Unit = {
// eager initialization, but not from constructor
self ! CassandraJournal.Init
}
override def receivePluginInternal: Receive = {
case WriteFinished(persistenceId, f) =>
writeInProgress.remove(persistenceId, f)
case DeleteFinished(persistenceId, toSequenceNr, result) =>
log.debug("Delete finished for persistence id [{}] to [{}] result [{}]", persistenceId, toSequenceNr, result)
pendingDeletes.get(persistenceId) match {
case null =>
log.error(
"Delete finished but not in pending. Please raise a bug with logs. PersistenceId: [{}]",
persistenceId)
case Nil =>
log.error(
"Delete finished but not in pending (empty). Please raise a bug with logs. PersistenceId: [{}]",
persistenceId)
case current :: tail =>
current.p.complete(result)
tail match {
case Nil =>
pendingDeletes.remove(persistenceId)
case next :: _ =>
pendingDeletes.put(persistenceId, tail)
delete(next.pid, next.toSequenceNr)
}
}
case CassandraJournal.Init =>
// try initialize early, to be prepared for first real request
preparedWriteMessage
preparedWriteMessageWithMeta
preparedSelectMessages
preparedSelectHighestSequenceNr
if (settings.journalSettings.supportAllPersistenceIds)
preparedInsertIntoAllPersistenceIds
if (settings.journalSettings.supportDeletes) {
preparedDeleteMessages
preparedSelectDeletedTo
preparedInsertDeletedTo
}
queries.initialize()
if (settings.eventsByTagSettings.eventsByTagEnabled) {
taggedPreparedStatements.init()
}
case DeleteAllEvents(persistenceId, neverUsePersistenceIdAgain) =>
val result = asyncDeleteMessagesTo(persistenceId, Long.MaxValue)
val result2: Future[Done] =
if (neverUsePersistenceIdAgain)
result
.flatMap(_ => deleteDeletedToSeqNr(persistenceId))
.flatMap(_ => deleteFromAllPersistenceIds(persistenceId))
else result.map(_ => Done)(ExecutionContexts.parasitic)
result2.pipeTo(sender())
case HealthCheckQuery =>
session.selectOne(healthCheckCql).map(_ => HealthCheckResponse)(ExecutionContexts.parasitic).pipeTo(sender())
}
override def asyncWriteMessages(messages: Seq[AtomicWrite]): Future[Seq[Try[Unit]]] = {
// we need to preserve the order / size of this sequence even though we don't map
// AtomicWrites 1:1 with a C* insert
//
// We must NOT catch serialization exceptions here because rejections will cause
// holes in the sequence number series and we use the sequence numbers to detect
// missing (delayed) events in the eventByTag query.
//
// Note that we assume that all messages have the same persistenceId, which is
// the case for Akka 2.4.2.
def serialize(aw: Seq[(PersistentRepr, UUID)]): Future[SerializedAtomicWrite] = {
val serializedEventsFut: Future[Seq[Serialized]] =
Future.sequence(aw.map {
case (pr, uuid) =>
val (pr2, tags) = pr.payload match {
case Tagged(payload, ts) =>
(pr.withPayload(payload), ts)
case _ =>
(pr, Set.empty[String])
}
serializeEvent(pr2, tags, uuid, settings.eventsByTagSettings.bucketSize, serialization, context.system)
})
serializedEventsFut.map { serializedEvents =>
SerializedAtomicWrite(aw.head._1.persistenceId, serializedEvents)
}
}
val writesWithUuids: Seq[Seq[(PersistentRepr, UUID)]] =
messages.map(aw => aw.payload.map(pr => (pr, generateUUID(pr))))
val writeInProgressForPersistentId = Promise[Done]()
val pid = messages.head.persistenceId
writeInProgress.put(pid, writeInProgressForPersistentId.future)
val toReturn: Future[Nil.type] = Future.sequence(writesWithUuids.map(w => serialize(w))).flatMap {
(serialized: Seq[SerializedAtomicWrite]) =>
val result: Future[Any] =
if (messages.map(_.payload.size).sum <= journalSettings.maxMessageBatchSize) {
// optimize for the common case
writeMessages(serialized)
} else {
// if presistAll was used, single AtomicWrite can already contain complete batch, so we need to regroup writes correctly
val groups: List[List[SerializedAtomicWrite]] = groupedWrites(serialized.toList.reverse, Nil, Nil)
// execute the groups in sequence
def rec(todo: List[List[SerializedAtomicWrite]]): Future[Any] =
todo match {
case write :: remainder =>
writeMessages(write).flatMap(_ => rec(remainder))
case Nil => FutureUnit
}
rec(groups)
}
// The tag writer keeps retrying but will drop writes for a persistent actor when it restarts
// due to this failing
result.flatMap { _ =>
tagWrites match {
case Some(t) =>
implicit val timeout: Timeout = Timeout(settings.eventsByTagSettings.tagWriteTimeout)
t.ask(extractTagWrites(serialized)).map(_ => Nil)(ExecutionContexts.parasitic)
case None => Future.successful(Nil)
}
}
}
// if the write fails still need to remove state from the map
toReturn.onComplete { _ =>
sendWriteFinished(pid, writeInProgressForPersistentId)
}
toReturn
}
// Regroup batches by payload size
@tailrec
private def groupedWrites(
reversed: List[SerializedAtomicWrite],
currentGroup: List[SerializedAtomicWrite],
grouped: List[List[SerializedAtomicWrite]]): List[List[SerializedAtomicWrite]] = reversed match {
case Nil => (currentGroup +: grouped).filterNot(_.isEmpty)
case x :: xs if currentGroup.size + x.payload.size < journalSettings.maxMessageBatchSize =>
groupedWrites(xs, x +: currentGroup, grouped)
case x :: xs => groupedWrites(xs, List(x), currentGroup +: grouped)
}
private def sendWriteFinished(pid: String, writeInProgressForPid: Promise[Done]): Unit = {
self ! WriteFinished(pid, writeInProgressForPid.future)
writeInProgressForPid.success(Done)
}
/**
* UUID generation is deliberately externalized to allow subclasses to customize the time based uuid for special cases.
* see https://discuss.lightbend.com/t/akka-persistence-cassandra-events-by-tags-bucket-size-based-on-time-vs-burst-load/1411 and make sure you understand the risk of doing this wrong.
*/
protected def generateUUID(pr: PersistentRepr): UUID = Uuids.timeBased()
private def extractTagWrites(serialized: Seq[SerializedAtomicWrite]): BulkTagWrite = {
if (serialized.isEmpty) BulkTagWrite(Nil, Nil)
else if (serialized.size == 1 && serialized.head.payload.size == 1) {
// optimization for one single event, which is the typical case
val s = serialized.head.payload.head
if (s.tags.isEmpty) BulkTagWrite(Nil, s :: Nil)
else BulkTagWrite(s.tags.map(tag => TagWrite(tag, s :: Nil)).toList, Nil)
} else {
val messagesByTag: Map[String, Seq[Serialized]] =
serialized.flatMap(_.payload).flatMap(s => s.tags.map((_, s))).groupBy(_._1).map {
case (tag, messages) => (tag, messages.map(_._2))
}
val messagesWithoutTag =
for {
a <- serialized
b <- a.payload
if b.tags.isEmpty
} yield b
val writesWithTags: immutable.Seq[TagWrite] = messagesByTag.map {
case (tag, writes) => TagWrite(tag, writes)
}.toList
BulkTagWrite(writesWithTags, messagesWithoutTag)
}
}
private def writeMessages(atomicWrites: Seq[SerializedAtomicWrite]): Future[Unit] = {
// insert into the all_persistence_ids table for the first event, used by persistenceIds query
require(atomicWrites.nonEmpty)
require(atomicWrites.head.payload.nonEmpty)
val allPersistenceId =
if (settings.journalSettings.supportAllPersistenceIds && atomicWrites.head.payload.head.sequenceNr == 1L)
preparedInsertIntoAllPersistenceIds.map(_.bind(atomicWrites.head.persistenceId)).flatMap(execute(_))
else
FutureUnit
val boundStatements: Seq[Future[BoundStatement]] = statementGroup(atomicWrites)
allPersistenceId.flatMap { _ =>
boundStatements.size match {
case 1 =>
boundStatements.head.flatMap(execute(_))
case 0 => FutureUnit
case _ =>
Future.sequence(boundStatements).flatMap { stmts =>
executeBatch(batch => stmts.foldLeft(batch) { case (acc, next) => acc.add(next) })
}
}
}
}
private def statementGroup(atomicWrites: Seq[SerializedAtomicWrite]): Seq[Future[BoundStatement]] = {
val maxPnr = partitionNr(atomicWrites.last.payload.last.sequenceNr, journalSettings.targetPartitionSize)
val firstSeq = atomicWrites.head.payload.head.sequenceNr
val minPnr = partitionNr(firstSeq, journalSettings.targetPartitionSize)
val persistenceId: String = atomicWrites.head.persistenceId
val all = atomicWrites.flatMap(_.payload)
// reading assumes sequence numbers are in the right partition or partition + 1
// even if we did allow this it would perform terribly as large C* batches are not good
require(
maxPnr - minPnr <= 1,
"Do not support AtomicWrites that span 3 partitions. Keep AtomicWrites <= max partition size.")
val writes: Seq[Future[BoundStatement]] = all.map { (m: Serialized) =>
// using two separate statements with or without the meta data columns because
// then users doesn't have to alter table and add the new columns if they don't use
// the meta data feature
val stmt =
if (m.meta.isDefined) preparedWriteMessageWithMeta
else preparedWriteMessage
stmt.map { stmt =>
val bs = stmt
.bind()
.setString("persistence_id", persistenceId)
.setLong("partition_nr", maxPnr)
.setLong("sequence_nr", m.sequenceNr)
.setUuid("timestamp", m.timeUuid)
// Keeping as text for backward compatibility
.setString("timebucket", m.timeBucket.key.toString)
.setString("writer_uuid", m.writerUuid)
.setInt("ser_id", m.serId)
.setString("ser_manifest", m.serManifest)
.setString("event_manifest", m.eventAdapterManifest)
.setByteBuffer("event", m.serialized)
.setSet("tags", m.tags.asJava, classOf[String])
// meta data, if any
m.meta
.map(meta => {
bs.setInt("meta_ser_id", meta.serId)
.setString("meta_ser_manifest", meta.serManifest)
.setByteBuffer("meta", meta.serialized)
})
.getOrElse(bs)
}
}
writes
}
/**
* It is assumed that this is only called during a replay and if fromSequenceNr == highest
* then asyncReplayMessages won't be called. In that case the tag progress is updated
* in here rather than during replay messages.
*/
override def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = {
log.debug("[{}] asyncReadHighestSequenceNr [{}] [{}]", persistenceId, fromSequenceNr, sender())
val highestSequenceNr = writeInProgress.get(persistenceId) match {
case null =>
asyncReadHighestSequenceNrInternal(persistenceId, fromSequenceNr)
case f =>
f.flatMap(_ => asyncReadHighestSequenceNrInternal(persistenceId, fromSequenceNr))
}
val toReturn = tagRecovery match {
case Some(tr) =>
// This relies on asyncReadHighestSequenceNr having the correct sender()
// No other calls into the async journal have this as they are called from Future callbacks
val persistentActor = sender()
for {
seqNr <- highestSequenceNr
_ <- tagRecovery.get.sendPersistentActorStarting(persistenceId, persistentActor)
_ <- if (seqNr == fromSequenceNr && seqNr != 0) {
log.debug(
"[{}] snapshot is current so replay won't be required. Calculating tag progress now",
persistenceId)
val scanningSeqNrFut = tr.tagScanningStartingSequenceNr(persistenceId)
for {
tp <- tr.lookupTagProgress(persistenceId)
_ <- tr.setTagProgress(persistenceId, tp)
scanningSeqNr <- scanningSeqNrFut
_ <- sendPreSnapshotTagWrites(scanningSeqNr, fromSequenceNr, persistenceId, Long.MaxValue, tp, tr)
} yield seqNr
} else if (seqNr == 0) {
log.debug("[{}] New pid. Sending blank tag progress. [{}]", persistenceId, persistentActor)
tr.setTagProgress(persistenceId, Map.empty)
} else {
FutureUnit
}
} yield seqNr
case None =>
highestSequenceNr
}
toReturn.onComplete { highestSeq =>
log.debug("asyncReadHighestSequenceNr {} returning {}", persistenceId, highestSeq)
}
toReturn
}
private def asyncReadHighestSequenceNrInternal(persistenceId: String, fromSequenceNr: Long): Future[Long] = {
asyncHighestDeletedSequenceNumber(persistenceId).flatMap { h =>
asyncFindHighestSequenceNr(
persistenceId,
math.max(fromSequenceNr, h),
settings.journalSettings.targetPartitionSize)
}
}
/**
* Not thread safe. Assumed to only be called from the journal actor.
* However, unlike asyncWriteMessages it can be called before the previous Future completes
*/
override def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long): Future[Unit] = {
// TODO could "optimize" away deletes that overlap?
pendingDeletes.get(persistenceId) match {
case null =>
log.debug("[{}] No outstanding delete. Sequence nr [{}]", persistenceId, toSequenceNr)
// fast path, no outstanding deletes for this persistenceId
val p = Promise[Unit]()
pendingDeletes.put(persistenceId, List(PendingDelete(persistenceId, toSequenceNr, p)))
delete(persistenceId, toSequenceNr)
p.future
case otherDeletes =>
if (otherDeletes.length > settings.journalSettings.maxConcurrentDeletes) {
log.error(
"[{}] Over [{}] outstanding deletes. Failing delete",
persistenceId,
settings.journalSettings.maxConcurrentDeletes)
Future.failed(new RuntimeException(
s"Over ${settings.journalSettings.maxConcurrentDeletes} outstanding deletes for persistenceId $persistenceId"))
} else {
log.debug(
"[{}] outstanding delete. Delete to seqNr [{}] will be scheduled after previous one finished.",
persistenceId,
toSequenceNr)
val p = Promise[Unit]()
pendingDeletes.put(persistenceId, otherDeletes :+ PendingDelete(persistenceId, toSequenceNr, p))
p.future
}
}
}
private def delete(persistenceId: String, toSequenceNr: Long): Future[Unit] = {
def physicalDelete(lowestPartition: Long, highestPartition: Long, toSeqNr: Long): Future[Done] = {
session.serverMetaData.flatMap { meta =>
if (meta.isVersion2 || settings.cosmosDb) {
physicalDelete2xCompat(lowestPartition, highestPartition, toSeqNr)
} else {
val deleteResult =
Future.sequence((lowestPartition to highestPartition).map { partitionNr =>
val boundDeleteMessages =
preparedDeleteMessages.map(_.bind(persistenceId, partitionNr: JLong, toSeqNr: JLong))
boundDeleteMessages.flatMap(execute(_))
})
deleteResult.failed.foreach { e =>
log.warning(
"Unable to complete deletes for persistence id {}, toSequenceNr {}. " +
"The plugin will continue to function correctly but you will need to manually delete the old messages. " +
"Caused by: [{}: {}]",
persistenceId,
toSequenceNr,
e.getClass.getName,
e.getMessage)
}
deleteResult.map(_ => Done)(ExecutionContexts.parasitic)
}
}
}
def physicalDelete2xCompat(
lowestPartition: TagPidSequenceNr,
highestPartition: TagPidSequenceNr,
toSeqNr: TagPidSequenceNr): Future[Done] = {
def asyncDeleteMessages(partitionNr: TagPidSequenceNr, messageIds: Seq[MessageId]): Future[Unit] = {
val boundStatements = messageIds.map(mid =>
preparedDeleteMessages.map(_.bind(mid.persistenceId, partitionNr: JLong, mid.sequenceNr: JLong)))
Future.sequence(boundStatements).flatMap { stmts =>
executeBatch(batch => stmts.foldLeft(batch) { case (acc, next) => acc.add(next) })
}
}
val partitionInfos = (lowestPartition to highestPartition).map(partitionInfo(persistenceId, _, toSeqNr))
val deleteResult =
Future.sequence(partitionInfos.map(future =>
future.flatMap(pi => {
Future.sequence((pi.minSequenceNr to pi.maxSequenceNr).grouped(journalSettings.maxMessageBatchSize).map {
group =>
{
val groupDeleteResult =
asyncDeleteMessages(pi.partitionNr, group.map(MessageId(persistenceId, _)))
groupDeleteResult.failed.foreach { e =>
log.warning(
s"Unable to complete deletes for persistence id {}, toSequenceNr {}. " +
"The plugin will continue to function correctly but you will need to manually delete the old messages. " +
"Caused by: [{}: {}]",
persistenceId,
toSequenceNr,
e.getClass.getName,
e.getMessage)
}
groupDeleteResult
}
})
})))
deleteResult.map(_ => Done)(ExecutionContexts.parasitic)
}
// Deletes the events by inserting into the metadata table deleted_to and physically deletes the rows.
def logicalAndPhysicalDelete(highestDeletedSequenceNumber: Long, highestSequenceNr: Long): Future[Done] = {
val lowestPartition = partitionNr(highestDeletedSequenceNumber + 1, journalSettings.targetPartitionSize)
val toSeqNr = math.min(toSequenceNr, highestSequenceNr)
val highestPartition = partitionNr(toSeqNr, journalSettings.targetPartitionSize) + 1 // may have been moved to the next partition
val logicalDelete =
if (toSeqNr <= highestDeletedSequenceNumber) {
// already deleted same or higher sequence number, don't update highestDeletedSequenceNumber,
// but perform the physical delete (again), may be a retry request
FutureUnit
} else {
val boundInsertDeletedTo =
preparedInsertDeletedTo.map(_.bind(persistenceId, toSeqNr: JLong))
boundInsertDeletedTo.flatMap(execute)
}
logicalDelete.flatMap(_ => physicalDelete(lowestPartition, highestPartition, toSeqNr))
}
val deleteResult = for {
highestDeletedSequenceNumber <- asyncHighestDeletedSequenceNumber(persistenceId)
highestSequenceNr <- {
// MaxValue may be used as magic value to delete all events without specifying actual toSequenceNr
if (toSequenceNr == Long.MaxValue)
asyncFindHighestSequenceNr(persistenceId, highestDeletedSequenceNumber, journalSettings.targetPartitionSize)
else
Future.successful(toSequenceNr)
}
_ <- logicalAndPhysicalDelete(highestDeletedSequenceNumber, highestSequenceNr)
} yield ()
// Kick off any pending deletes when finished.
deleteResult.onComplete { result =>
self ! DeleteFinished(persistenceId, toSequenceNr, result)
}
deleteResult
}
private def deleteDeletedToSeqNr(persistenceId: String): Future[Done] = {
session.executeWrite(statements.journalStatements.deleteDeletedTo, persistenceId).map(_ => Done)
}
private def deleteFromAllPersistenceIds(persistenceId: String): Future[Done] = {
session.executeWrite(statements.journalStatements.deleteFromAllPersistenceIds, persistenceId).map(_ => Done)
}
private def partitionInfo(persistenceId: String, partitionNr: Long, maxSequenceNr: Long): Future[PartitionInfo] = {
val boundSelectHighestSequenceNr = preparedSelectHighestSequenceNr.map(_.bind(persistenceId, partitionNr: JLong))
boundSelectHighestSequenceNr
.flatMap(selectOne)
.map(row =>
row
.map(s =>
PartitionInfo(partitionNr, minSequenceNr(partitionNr), math.min(s.getLong("sequence_nr"), maxSequenceNr)))
.getOrElse(PartitionInfo(partitionNr, minSequenceNr(partitionNr), -1)))
}
private def asyncHighestDeletedSequenceNumber(persistenceId: String): Future[Long] = {
preparedSelectDeletedTo match {
case Some(pstmt) =>
val boundSelectDeletedTo = pstmt.map(_.bind(persistenceId))
boundSelectDeletedTo.flatMap(selectOne).map(rowOption => rowOption.map(_.getLong("deleted_to")).getOrElse(0))
case None =>
Future.successful(0L)
}
}
private def asyncFindHighestSequenceNr(
persistenceId: String,
fromSequenceNr: Long,
partitionSize: Long): Future[Long] = {
def find(currentPnr: Long, currentSnr: Long, foundEmptyPartition: Boolean): Future[Long] = {
// if every message has been deleted and thus no sequence_nr the driver gives us back 0 for "null" :(
val boundSelectHighestSequenceNr = preparedSelectHighestSequenceNr.map(ps => {
val bound = ps.bind(persistenceId, currentPnr: JLong)
bound
})
boundSelectHighestSequenceNr
.flatMap(selectOne)
.map { rowOption =>
rowOption.map(_.getLong("sequence_nr"))
}
.flatMap {
case None | Some(0) =>
// never been to this partition, query one more partition because AtomicWrite can span (skip)
// one entire partition
// Some(0) when old schema with static used column, everything deleted in this partition
if (foundEmptyPartition) Future.successful(currentSnr)
else find(currentPnr + 1, currentSnr, foundEmptyPartition = true)
case Some(nextHighest) =>
find(currentPnr + 1, nextHighest, foundEmptyPartition = false)
}
}
find(partitionNr(fromSequenceNr, partitionSize), fromSequenceNr, foundEmptyPartition = false)
}
private def executeBatch(body: BatchStatement => BatchStatement): Future[Unit] = {
var batch =
new BatchStatementBuilder(BatchType.UNLOGGED).build().setExecutionProfileName(journalSettings.writeProfile)
batch = body(batch)
session.underlying().flatMap(_.executeAsync(batch).asScala).map(_ => ())
}
private def selectOne[T <: Statement[T]](stmt: Statement[T]): Future[Option[Row]] = {
session.selectOne(stmt.setExecutionProfileName(journalSettings.readProfile))
}
private def minSequenceNr(partitionNr: Long): Long =
partitionNr * journalSettings.targetPartitionSize + 1
private def execute[T <: Statement[T]](stmt: Statement[T]): Future[Unit] = {
session.executeWrite(stmt.setExecutionProfileName(journalSettings.writeProfile)).map(_ => ())
}
// TODO this serialises and re-serialises the messages for fixing tag_views
// Could have an events by persistenceId stage that has the raw payload
override def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(
replayCallback: PersistentRepr => Unit): Future[Unit] = {
log.debug("[{}] asyncReplayMessages from [{}] to [{}]", persistenceId, fromSequenceNr, toSequenceNr)
tagRecovery match {
case Some(tr) =>
val recoveryPrep: Future[Map[String, TagProgress]] = {
val scanningSeqNrFut = tr.tagScanningStartingSequenceNr(persistenceId)
for {
tp <- tr.lookupTagProgress(persistenceId)
_ <- tr.setTagProgress(persistenceId, tp)
scanningSeqNr <- scanningSeqNrFut
_ <- sendPreSnapshotTagWrites(scanningSeqNr, fromSequenceNr, persistenceId, max, tp, tr)
} yield tp
}
Source
.futureSource(recoveryPrep.map((tp: Map[Tag, TagProgress]) => {
log.debug(
"[{}] starting recovery with tag progress: [{}]. From [{}] to [{}]",
persistenceId,
tp,
fromSequenceNr,
toSequenceNr)
queries
.eventsByPersistenceId(
persistenceId,
fromSequenceNr,
toSequenceNr,
max,
None,
settings.journalSettings.readProfile,
"asyncReplayMessages",
extractor = Extractors.taggedPersistentRepr(eventDeserializer, serialization))
.mapAsync(1)(tr.sendMissingTagWrite(tp))
}))
.map(te => queries.mapEvent(te.pr))
.runForeach(replayCallback)
.map(_ => ())
case None =>
queries
.eventsByPersistenceId(
persistenceId,
fromSequenceNr,
toSequenceNr,
max,
None,
settings.journalSettings.readProfile,
"asyncReplayMessages",
extractor = Extractors.persistentRepr(eventDeserializer, serialization))
.map(queries.mapEvent)
.runForeach(replayCallback)
.map(_ => ())
}
}
private def sendPreSnapshotTagWrites(
minProgressNr: Long,
fromSequenceNr: Long,
pid: String,
max: Long,
tp: Map[Tag, TagProgress],
tr: CassandraTagRecovery): Future[Done] = {
if (minProgressNr < fromSequenceNr) {
val scanTo = fromSequenceNr - 1
log.debug(
"[{}], Scanning events before snapshot to recover tag_views: From: [{}] to: [{}]",
pid,
minProgressNr,
scanTo)
queries
.eventsByPersistenceId(
pid,
minProgressNr,
scanTo,
max,
None,
settings.journalSettings.readProfile,
"asyncReplayMessagesPreSnapshot",
Extractors.optionalTaggedPersistentRepr(eventDeserializer, serialization))
.mapAsync(1) { t =>
t.tagged match {
case OptionVal.Some(tpr) =>
tr.sendMissingTagWrite(tp)(tpr)
case OptionVal.None => FutureDone // no tags, skip
}
}
.runWith(Sink.ignore)
} else {
log.debug(
"[{}] Recovery is starting before the latest tag writes tag progress. Min progress [{}]. From sequence nr of recovery: [{}]",
pid,
minProgressNr,
fromSequenceNr)
FutureDone
}
}
}
/**
* INTERNAL API
*/
@InternalApi private[pekko] object CassandraJournal {
private[pekko] type Tag = String
private[pekko] type PersistenceId = String
private[pekko] type SequenceNr = Long
private[pekko] type TagPidSequenceNr = Long
private case object Init
sealed trait CleanupCommand
final case class DeleteAllEvents(persistenceId: String, neverUsePersistenceIdAgain: Boolean) extends CleanupCommand
private case class WriteFinished(pid: String, f: Future[Done]) extends NoSerializationVerificationNeeded
private case class DeleteFinished(pid: String, toSequenceNr: Long, f: Try[Unit])
extends NoSerializationVerificationNeeded
private case class PendingDelete(pid: String, toSequenceNr: Long, p: Promise[Unit])
extends NoSerializationVerificationNeeded
private case class SerializedAtomicWrite(persistenceId: String, payload: Seq[Serialized])
private[pekko] case class Serialized(
persistenceId: String,
sequenceNr: Long,
serialized: ByteBuffer,
tags: Set[String],
eventAdapterManifest: String,
serManifest: String,
serId: Int,
writerUuid: String,
meta: Option[SerializedMeta],
timeUuid: UUID,
timeBucket: TimeBucket) {
// never log serialized byte buffer
override def toString: PersistenceId = s"Serialized($persistenceId, $sequenceNr, $timeBucket)"
}
private[pekko] case class SerializedMeta(serialized: ByteBuffer, serManifest: String, serId: Int)
private case class PartitionInfo(partitionNr: Long, minSequenceNr: Long, maxSequenceNr: Long)
private case class MessageId(persistenceId: String, sequenceNr: Long)
sealed trait HealthCheck
case object HealthCheckQuery extends HealthCheck
case object HealthCheckResponse extends HealthCheck
final case class DeserializedEvent(event: Any, meta: OptionVal[Any])
class EventDeserializer(system: ActorSystem) {
private val log = Logging(system, classOf[CassandraJournal])
private val serialization = SerializationExtension(system)
val columnDefinitionCache = new ColumnDefinitionCache
def deserializeEvent(row: Row, async: Boolean)(implicit ec: ExecutionContext): Future[DeserializedEvent] =
try {
def meta: OptionVal[AnyRef] = {
if (columnDefinitionCache.hasMetaColumns(row)) {
row.getByteBuffer("meta") match {
case null =>
OptionVal.None // no meta data
case metaBytes =>
// has meta data, wrap in EventWithMetaData
val metaSerId = row.getInt("meta_ser_id")
val metaSerManifest = row.getString("meta_ser_manifest")
serialization.deserialize(Bytes.getArray(metaBytes), metaSerId, metaSerManifest) match {
case Success(m) => OptionVal.Some(m)
case Failure(ex) =>
log.warning(
"Deserialization of event metadata failed (pid: [{}], seq_nr: [{}], meta_ser_id: [{}], meta_ser_manifest: [{}], ignoring metadata content. Exception: {}",
Array(
row.getString("persistence_id"),
row.getLong("sequence_nr"),
metaSerId,
metaSerManifest,
ex.toString))
OptionVal.None
}
}
} else {
// for backwards compatibility, when table was not altered, meta columns not added
OptionVal.None // no meta data
}
}
val bytes = Bytes.getArray(row.getByteBuffer("event"))
val serId = row.getInt("ser_id")
val manifest = row.getString("ser_manifest")
(serialization.serializerByIdentity.get(serId) match {
case Some(asyncSerializer: AsyncSerializer) =>
Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () =>
asyncSerializer.fromBinaryAsync(bytes, manifest)
}
case _ =>
def deserializedEvent: AnyRef =
// Serialization.deserialize adds transport info
serialization.deserialize(bytes, serId, manifest).get
if (async) Future(deserializedEvent)
else Future.successful(deserializedEvent)
}).map(event => DeserializedEvent(event, meta))(ExecutionContexts.parasitic)
} catch {
case NonFatal(e) => Future.failed(e)
}
}
}