-
Notifications
You must be signed in to change notification settings - Fork 8
/
BackupClientInterface.scala
601 lines (548 loc) · 26.1 KB
/
BackupClientInterface.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
package io.aiven.guardian.kafka.backup
import com.typesafe.scalalogging.LazyLogging
import io.aiven.guardian.kafka.Errors
import io.aiven.guardian.kafka.backup.configs.{Compression => CompressionConfig, _}
import io.aiven.guardian.kafka.codecs.Circe._
import io.aiven.guardian.kafka.models.BackupObjectMetadata
import io.aiven.guardian.kafka.models.CompressionType
import io.aiven.guardian.kafka.models.Gzip
import io.aiven.guardian.kafka.models.ReducedConsumerRecord
import io.circe.syntax._
import org.apache.pekko
import scala.annotation.nowarn
import scala.concurrent.ExecutionContext
import scala.concurrent.Future
import scala.jdk.DurationConverters._
import java.time._
import java.time.format.DateTimeFormatter
import java.time.temporal._
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.stream.scaladsl._
import pekko.util.ByteString
/** An interface for a template on how to backup a Kafka Stream into some data storage
* @tparam T
* The underlying `kafkaClientInterface` type
*/
trait BackupClientInterface[T <: KafkaConsumerInterface] extends LazyLogging {
implicit val kafkaClientInterface: T
implicit val backupConfig: Backup
implicit val system: ActorSystem
/** An element from the original record
*/
private[backup] sealed trait RecordElement
private[backup] case class Element(reducedConsumerRecord: ReducedConsumerRecord,
context: kafkaClientInterface.CursorContext
) extends RecordElement
private[backup] case object End extends RecordElement
private[backup] sealed trait ByteStringContext {
val context: kafkaClientInterface.CursorContext
}
private[backup] case class Start(override val context: kafkaClientInterface.CursorContext, key: String)
extends ByteStringContext
private[backup] case class Tail(override val context: kafkaClientInterface.CursorContext) extends ByteStringContext
case class StateDetails(state: State, backupObjectMetadata: BackupObjectMetadata)
case class PreviousState(stateDetails: StateDetails, previousKey: String)
case class UploadStateResult(current: Option[StateDetails], previous: Option[PreviousState])
object UploadStateResult {
val empty: UploadStateResult = UploadStateResult(None, None)
}
/** Override this type to define the result of backing up data to a datasource
*/
type BackupResult
/** Override this type to define the result of calculating the previous state (if it exists)
*/
type State
import BackupClientInterface._
/** Override this method to define how to retrieve the current state of any unfinished backups.
* @param key
* The object key or filename for what is currently being backed up
* @return
* A [[scala.concurrent.Future]] with a [[UploadStateResult]] data structure that optionally contains the state
* associated with `key` along with the previous latest state before `key` (if it exists)
*/
def getCurrentUploadState(key: String): Future[UploadStateResult]
/** A sink that is executed whenever a previously existing Backup needs to be terminated and closed. Generally
* speaking this [[pekko.stream.scaladsl.Sink]] is similar to the `backupToStorageSink` except that
* `kafkaClientInterface.CursorContext` is not required since no Kafka messages are being written.
*
* Note that the terminate refers to the fact that this Sink is executed with a `null]`
* [[pekko.stream.scaladsl.Source]] which when written to an already existing unfinished backup terminates the
* containing JSON array so that it becomes valid parsable JSON.
* @param previousState
* A data structure containing both the [[State]] along with the associated key which you can refer to in order to
* define your [[pekko.stream.scaladsl.Sink]]
* @return
* A [[pekko.stream.scaladsl.Sink]] that points to an existing key defined by `previousState.previousKey`
*/
def backupToStorageTerminateSink(previousState: PreviousState): Sink[ByteString, Future[BackupResult]]
/** Override this method to define how to backup a [[pekko.util.ByteString]] combined with Kafka
* `kafkaClientInterface.CursorContext` to a `DataSource`
* @param key
* The object key or filename for what is being backed up
* @param currentState
* The current state if it exists. If this is empty then a new backup is being created with the associated `key`
* otherwise if this contains a [[State]] then the defined [[pekko.stream.scaladsl.Sink]] needs to handle resuming
* a previously unfinished backup with that `key` by directly appending the [[pekko.util.ByteString]] data.
* @return
* A [[pekko.stream.scaladsl.Sink]] that given a [[pekko.util.ByteString]] (containing a single Kafka
* [[io.aiven.guardian.kafka.models.ReducedConsumerRecord]]) along with its `kafkaClientInterface.CursorContext`
* backs up the data to your data storage. The [[pekko.stream.scaladsl.Sink]] is also responsible for executing
* `kafkaClientInterface.commitCursor` when the data is successfully backed up
*/
def backupToStorageSink(key: String,
currentState: Option[State]
): Sink[(ByteString, kafkaClientInterface.CursorContext), Future[BackupResult]]
/** Override this method to define a zero vale that covers the case that occurs immediately when `SubFlow` has been
* split at `BackupStreamPosition.Start`. If you have difficulties defining an empty value for `BackupResult` then
* you can wrap it in an `Option` and just use `None` for the empty case
* @return
* An "empty" value that is used when a split `SubFlow` has just started
*/
def empty: () => Future[BackupResult]
private[backup] def calculateBackupStreamPositions(
sourceWithPeriods: SourceWithContext[(ReducedConsumerRecord, Long),
kafkaClientInterface.CursorContext,
kafkaClientInterface.Control
]
): Source[RecordElement, kafkaClientInterface.Control] =
sourceWithPeriods.asSource
.prefixAndTail(2)
// This algorithm only works with Source's that have 2 or more elements
.flatMapConcat {
case (Seq(
((firstReducedConsumerRecord, firstDivision), firstContext),
((secondReducedConsumerRecord, secondDivision), secondContext)
),
rest
) =>
val all = Source
.combine(
Source(
List(
((firstReducedConsumerRecord, firstDivision), firstContext),
((secondReducedConsumerRecord, secondDivision), secondContext)
)
),
rest
)(Concat(_))
val withDivisions =
all
.sliding(2)
.map {
case Seq(((_, beforeDivisions), _), ((afterReducedConsumerRecord, afterDivisions), afterContext)) =>
if (isAtBoundary(beforeDivisions, afterDivisions))
List(
End,
Element(afterReducedConsumerRecord, afterContext)
)
else
List(Element(afterReducedConsumerRecord, afterContext))
case rest =>
throw Errors.UnhandledStreamCase(rest)
}
.mapConcat(identity)
Source.combine(
Source.single(Element(firstReducedConsumerRecord, firstContext)),
withDivisions
)(Concat(_))
// This case only occurs if a Source has a single element so we just directly return it
case (Seq(((singleReducedConsumerRecord, _), singleContext)), _) =>
Source.single(Element(singleReducedConsumerRecord, singleContext))
case (rest, _) =>
throw Errors.UnhandledStreamCase(rest)
}
private[backup] def sourceWithPeriods(
source: Source[(OffsetDateTime, (ReducedConsumerRecord, kafkaClientInterface.CursorContext)),
kafkaClientInterface.Control
]
): SourceWithContext[(ReducedConsumerRecord, Long),
kafkaClientInterface.CursorContext,
kafkaClientInterface.Control
] = SourceWithContext.fromTuples(source.map { case (firstTimestamp, (record, context)) =>
val period = calculateNumberOfPeriodsFromTimestamp(firstTimestamp, backupConfig.timeConfiguration, record)
((record, period), context)
})
private[backup] def sourceWithFirstRecord
: Source[(OffsetDateTime, (ReducedConsumerRecord, kafkaClientInterface.CursorContext)),
kafkaClientInterface.Control
] =
kafkaClientInterface.getSource.asSource.prefixAndTail(1).flatMapConcat { case (head, rest) =>
head.headOption match {
case Some((firstReducedConsumerRecord, firstCursorContext)) =>
val firstTimestamp = firstReducedConsumerRecord.toOffsetDateTime
Source.combine(
Source.single((firstTimestamp, (firstReducedConsumerRecord, firstCursorContext))),
rest.map { case (reducedConsumerRecord, context) =>
(firstTimestamp, (reducedConsumerRecord, context))
}
)(Concat(_))
case None => throw Errors.ExpectedStartOfSource
}
}
/** Transforms a sequence of [[RecordElement]]'s into a ByteString so that it can be persisted into the data storage
*
* @param sourceElements
* A sequence of [[RecordElement]]'s as a result of `sliding(2)`
* @return
* a [[ByteString]] ready to be persisted along with the original context form the [[RecordElement]]
*/
private[backup] def transformReducedConsumerRecords(sourceElements: Seq[RecordElement]) = {
val stringWithContext = sourceElements match {
case Seq(Element(reducedConsumerRecord, context)) =>
// Happens in Sentinel case that is explicitly called at start of stream OR when a stream is interrupted by the user
// in which case stream needs to be terminated with `null]` in order to be valid
List((s"${reducedConsumerRecordAsString(reducedConsumerRecord)},", Some(context)))
case Seq(Element(firstReducedConsumerRecord, firstContext),
Element(secondReducedConsumerRecord, secondContext)
) =>
List(
(s"${reducedConsumerRecordAsString(firstReducedConsumerRecord)},", Some(firstContext)),
(s"${reducedConsumerRecordAsString(secondReducedConsumerRecord)},", Some(secondContext))
)
case Seq(Element(reducedConsumerRecord, context), End) =>
List((s"${reducedConsumerRecordAsString(reducedConsumerRecord)}]", Some(context)))
case Seq(End) =>
List(("]", None))
case rest => throw Errors.UnhandledStreamCase(rest)
}
stringWithContext.map { case (string, context) => (ByteString(string), context) }
}
private[backup] def dropCommaFromEndOfJsonArray(byteString: ByteString) =
byteString.dropRight(1)
/** Applies the transformation to the first element of a Stream so that it starts of as a JSON array.
*
* @param element
* Starting [[Element]]
* @param key
* The current key being processed
* @param terminate
* Whether to immediately terminate the JSON array for single element in Stream case
* @return
* A [[List]] containing a single [[Start]] ready to be processed in the [[Sink]]
*/
private[backup] def transformFirstElement(element: Element, key: String, terminate: Boolean) =
SourceWithContext.fromTuples(Source(transformReducedConsumerRecords(List(element)).map {
case (byteString, Some(context)) =>
val bs =
if (terminate)
ByteString("[") ++ dropCommaFromEndOfJsonArray(byteString) ++ ByteString("]")
else
ByteString("[") ++ byteString
(bs, Start(context, key))
case _ =>
throw Errors.UnhandledStreamCase(List(element))
}))
/** Fixes the case where is an odd amount of elements in the stream
* @param head
* of stream as a result of `prefixAndTail`
* @param restSource
* of the stream as a result of `prefixAndTail`
* @return
* A [[List]] of ([[ByteString]], [[kafkaClientInterface.CursorContext]]) with the tail elements fixed up.
*/
private[backup] def transformTailingElement(
head: Seq[(ByteString, Option[kafkaClientInterface.CursorContext])],
restSource: Source[(ByteString, Option[kafkaClientInterface.CursorContext]), NotUsed]
) = {
val restTransformed = restSource
.sliding(2, step = 2)
.map {
case Seq((before, Some(context)), (after, None)) =>
List((dropCommaFromEndOfJsonArray(before) ++ after, context))
case Seq((before, Some(beforeContext)), (after, Some(afterContext))) =>
List((before, beforeContext), (after, afterContext))
case Seq((single, Some(context))) =>
List((single, context))
case rest =>
throw Errors.UnhandledStreamCase(rest)
}
head match {
case Seq((byteString, Some(cursorContext))) =>
Source.combine(
Source.single(List((byteString, cursorContext))),
restTransformed
)(Concat(_))
case rest =>
throw Errors.UnhandledStreamCase(rest)
}
}
private[backup] def compressContextFlow[CtxIn, CtxOut, Mat](
flowWithContext: FlowWithContext[ByteString, CtxIn, ByteString, CtxOut, Mat]
) =
backupConfig.compression match {
case Some(compression) =>
flowWithContext.unsafeDataVia(compressionFlow(compression))
case None => flowWithContext
}
private[backup] def compressContextSink[Ctx, Mat](sink: Sink[(ByteString, Ctx), Mat]) =
backupConfig.compression match {
case Some(compression) =>
FlowWithContext[ByteString, Ctx]
.unsafeDataVia(compressionFlow(compression))
.asFlow
.toMat(
sink
)(Keep.right)
case None => sink
}
private[backup] def skipCompressionOnAlreadyExistingUpload(start: Start, previousState: PreviousState) =
(backupConfig.compression, previousState.stateDetails.backupObjectMetadata.compression) match {
case (Some(compression), None) =>
val whichCompression = compression.`type`.pretty
logger.info(
s"Configured compression $whichCompression will apply on next upload, skipping compressing current upload"
)
backupToStorageSink(start.key, None)
.contramap[(ByteString, ByteStringContext)] { case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
}
case (None, None) =>
backupToStorageSink(start.key, None)
.contramap[(ByteString, ByteStringContext)] { case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
}
case (None, Some(_)) =>
logger.info(s"Compression has been configured to be disabled, this will apply on next upload")
FlowWithContext[ByteString, ByteStringContext]
// Since we don't persist any details on what the compression level is for a previously
// initiated upload lets just use the default compression level
.unsafeDataVia(compressionFlow(CompressionConfig(Gzip, None)))
.asFlow
.toMat(
backupToStorageSink(start.key, None)
.contramap[(ByteString, ByteStringContext)] { case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
}
)(Keep.right)
case (Some(CompressionConfig(Gzip, _)), Some(Gzip)) =>
compressContextSink(
backupToStorageSink(start.key, None)
.contramap[(ByteString, ByteStringContext)] { case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
}
)
}
/** Prepares the sink before it gets handed to `backupToStorageSink`
*/
private[backup] def prepareStartOfStream(uploadStateResult: UploadStateResult,
start: Start
): Sink[(ByteString, ByteStringContext), Future[BackupResult]] =
(uploadStateResult.previous, uploadStateResult.current) match {
case (Some(previous), None) =>
backupConfig.timeConfiguration match {
case _: PeriodFromFirst =>
skipCompressionOnAlreadyExistingUpload(start, previous)
case _: ChronoUnitSlice =>
logger.warn(
s"Detected previous backup using PeriodFromFirst however current configuration is now changed to ChronoUnitSlice. Object/file with an older key: ${start.key} may contain newer events than object/file with newer key: ${previous.previousKey}"
)
skipCompressionOnAlreadyExistingUpload(start, previous)
}
case (None, Some(current)) =>
backupConfig.timeConfiguration match {
case _: PeriodFromFirst =>
throw Errors.UnhandledStreamCase(List(current))
case _: ChronoUnitSlice =>
val baseFlow = FlowWithContext
.fromTuples(
Flow[(ByteString, ByteStringContext)]
.flatMapPrefix(1) {
case Seq((byteString, start: Start)) =>
val withoutStartOfJsonArray = byteString.drop(1)
Flow[(ByteString, ByteStringContext)].prepend(
Source.single((withoutStartOfJsonArray, start))
)
case _ => throw Errors.ExpectedStartOfSource
}
)
compressContextFlow(baseFlow).asFlow
.toMat(backupToStorageSink(start.key, Some(current.state)).contramap[(ByteString, ByteStringContext)] {
case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
})(Keep.right)
}
case (None, None) =>
compressContextSink(
backupToStorageSink(start.key, None)
.contramap[(ByteString, ByteStringContext)] { case (byteString, byteStringContext) =>
(byteString, byteStringContext.context)
}
)
case (Some(previous), Some(current)) =>
throw Errors.UnhandledStreamCase(List(previous.stateDetails, current))
}
/** The entire flow that involves reading from Kafka, transforming the data into JSON and then backing it up into a
* data source.
* @return
* The `KafkaClientInterface.Control` which depends on the implementation of `T` (typically this is used to control
* the underlying stream).
*/
def backup: RunnableGraph[kafkaClientInterface.MatCombineResult] = {
val withBackupStreamPositions = calculateBackupStreamPositions(sourceWithPeriods(sourceWithFirstRecord))
val split = withBackupStreamPositions.splitAfter {
case End => true
case _ => false
}
val substreams = split
.prefixAndTail(2)
.flatMapConcat {
case (Seq(only: Element, End), _) =>
// This case only occurs when you have a single element in a timeslice.
// We have to terminate immediately to create a JSON array with a single element
val key = calculateKey(only.reducedConsumerRecord.toOffsetDateTime,
backupConfig.timeConfiguration,
backupConfig.compression
)
transformFirstElement(only, key, terminate = true)
case (Seq(first: Element, second: Element), restOfReducedConsumerRecords) =>
val key = calculateKey(first.reducedConsumerRecord.toOffsetDateTime,
backupConfig.timeConfiguration,
backupConfig.compression
)
val firstSource = transformFirstElement(first, key, terminate = false)
val rest = Source.combine(
Source.single(second),
restOfReducedConsumerRecords
)(Concat(_))
val restTransformed = rest
.sliding(2, step = 2)
.map(transformReducedConsumerRecords)
.mapConcat(identity)
.prefixAndTail(1)
.flatMapConcat((transformTailingElement _).tupled)
.mapConcat(identity)
.map { case (byteString, context) => (byteString, Tail(context)) }
SourceWithContext.fromTuples(
Source.combine(
firstSource.asSource,
restTransformed
)(Concat(_))
)
case (Seq(only: Element), _) =>
// This case can also occur when user terminates the stream
val key = calculateKey(only.reducedConsumerRecord.toOffsetDateTime,
backupConfig.timeConfiguration,
backupConfig.compression
)
transformFirstElement(only, key, terminate = false)
case (rest, _) =>
throw Errors.UnhandledStreamCase(rest)
}
@nowarn("msg=method lazyInit in object Sink is deprecated")
val subFlowSink = substreams
.alsoTo(
// See https://stackoverflow.com/questions/68774425/combine-prefixandtail1-with-sink-lazysink-for-subflow-created-by-splitafter/68776660?noredirect=1#comment121558518_68776660
Sink.lazyInit(
{
case (_, start: Start) =>
implicit val ec: ExecutionContext = system.dispatcher
logger.debug(s"Calling getCurrentUploadState with key:${start.key}")
for {
uploadStateResult <- getCurrentUploadState(start.key)
_ = logger.debug(s"Received $uploadStateResult from getCurrentUploadState with key:${start.key}")
_ <- (uploadStateResult.previous, uploadStateResult.current) match {
case (Some(previous), None) =>
terminateSource(previous.stateDetails.backupObjectMetadata.compression)
.runWith(backupToStorageTerminateSink(previous))
.map(Some.apply)(ExecutionContext.parasitic)
case _ => Future.successful(None)
}
} yield prepareStartOfStream(uploadStateResult, start)
case _ => throw Errors.ExpectedStartOfSource
},
empty
)
)
.mergeSubstreamsWithParallelism(1)
subFlowSink.toMat(Sink.ignore)(kafkaClientInterface.matCombine)
}
}
object BackupClientInterface {
def reducedConsumerRecordAsString(reducedConsumerRecord: ReducedConsumerRecord): String =
io.circe.Printer.noSpaces.print(reducedConsumerRecord.asJson)
def formatOffsetDateTime(offsetDateTime: OffsetDateTime): String =
offsetDateTime.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME)
/** Calculate an object storage key or filename for a ReducedConsumerRecord
* @param offsetDateTime
* A given time
* @return
* A `String` that can be used either as some object key or a filename
*/
def calculateKey(offsetDateTime: OffsetDateTime,
timeConfiguration: TimeConfiguration,
maybeCompression: Option[CompressionConfig]
): String = {
val finalTime = timeConfiguration match {
case ChronoUnitSlice(chronoUnit) => offsetDateTime.truncatedTo(chronoUnit)
case _ => offsetDateTime
}
val extension = maybeCompression match {
case Some(CompressionConfig(Gzip, _)) => "json.gz"
case None => "json"
}
s"${BackupClientInterface.formatOffsetDateTime(finalTime)}.$extension"
}
/** Calculates whether we have rolled over a time period given number of divided periods.
* @param dividedPeriodsBefore
* The number of divided periods in the first element of the slide. -1 is used as a sentinel value to indicate the
* start of the stream
* @param dividedPeriodsAfter
* The number of divided periods in the second element of the slide
* @return
* `true` if we have hit a time boundary otherwise `false`
*/
def isAtBoundary(dividedPeriodsBefore: Long, dividedPeriodsAfter: Long): Boolean =
(dividedPeriodsBefore, dividedPeriodsAfter) match {
case (before, after) if after > before =>
true
case _ =>
false
}
protected def calculateNumberOfPeriodsFromTimestamp(initialTime: OffsetDateTime,
timeConfiguration: TimeConfiguration,
reducedConsumerRecord: ReducedConsumerRecord
): Long = {
val (period, finalInitialTime) = timeConfiguration match {
case PeriodFromFirst(duration) => (duration, initialTime)
case ChronoUnitSlice(chronoUnit) =>
(chronoUnit.getDuration.toScala, initialTime.truncatedTo(chronoUnit))
}
// TODO handle overflow?
ChronoUnit.MICROS.between(finalInitialTime, reducedConsumerRecord.toOffsetDateTime) / period.toMicros
}
/** Flattens an existing flow so that for each incoming element there is exactly one outputting element. Typically
* this is used in combination with `unsafeDataVia` so that your `FlowWithContext`/`SourceWithContext` doesn't get
* corrupted.
* @param flow
* An existing flow that you want to flatten
* @param zero
* A zero value which is used if the `flow` doesn't produce any elements
* @param foldFunc
* A function that determines how to concatenate a sequence
* @return
* A flow that will always produce exactly single output element for a given input element
*/
private[backup] def flattenFlow[T](flow: Flow[T, T, NotUsed], zero: T, foldFunc: (T, T) => T): Flow[T, T, NotUsed] =
Flow[T].flatMapConcat { single =>
Source
.fromMaterializer { case (mat, _) =>
Source.future(
Source.single(single).via(flow).runFold(zero)(foldFunc)(mat)
)
}
}
private[backup] def compressionFlow(compression: CompressionConfig) = compression match {
case CompressionConfig(Gzip, Some(level)) =>
flattenFlow[ByteString](Compression.gzip(level), ByteString.empty, _ ++ _)
case CompressionConfig(Gzip, None) =>
flattenFlow[ByteString](Compression.gzip, ByteString.empty, _ ++ _)
}
private[backup] def terminateSource(compression: Option[CompressionType]) = {
val baseSource = Source.single(ByteString("null]"))
compression match {
case Some(Gzip) => baseSource.via(Compression.gzip)
case None => baseSource
}
}
}