risingwave_connector/sink/
log_store.rs

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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::cmp::Ordering;
use std::collections::VecDeque;
use std::fmt::Debug;
use std::future::{poll_fn, Future};
use std::sync::Arc;
use std::task::Poll;
use std::time::Instant;

use await_tree::InstrumentAwait;
use futures::{TryFuture, TryFutureExt};
use risingwave_common::array::StreamChunk;
use risingwave_common::bail;
use risingwave_common::bitmap::Bitmap;
use risingwave_common::metrics::{LabelGuardedIntCounter, LabelGuardedIntGauge};
use risingwave_common::util::epoch::{EpochPair, INVALID_EPOCH};

pub type LogStoreResult<T> = Result<T, anyhow::Error>;
pub type ChunkId = usize;

#[derive(Debug, PartialEq, Copy, Clone)]
pub enum TruncateOffset {
    Chunk { epoch: u64, chunk_id: ChunkId },
    Barrier { epoch: u64 },
}

impl PartialOrd for TruncateOffset {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        let extract = |offset: &TruncateOffset| match offset {
            TruncateOffset::Chunk { epoch, chunk_id } => (*epoch, *chunk_id),
            TruncateOffset::Barrier { epoch } => (*epoch, usize::MAX),
        };
        let this = extract(self);
        let other = extract(other);
        this.partial_cmp(&other)
    }
}

impl TruncateOffset {
    pub fn next_chunk_id(&self) -> ChunkId {
        match self {
            TruncateOffset::Chunk { chunk_id, .. } => chunk_id + 1,
            TruncateOffset::Barrier { .. } => 0,
        }
    }

    pub fn epoch(&self) -> u64 {
        match self {
            TruncateOffset::Chunk { epoch, .. } | TruncateOffset::Barrier { epoch } => *epoch,
        }
    }

    pub fn check_next_offset(&self, next_offset: TruncateOffset) -> LogStoreResult<()> {
        if *self >= next_offset {
            bail!(
                "next offset {:?} should be later than current offset {:?}",
                next_offset,
                self
            )
        } else {
            Ok(())
        }
    }

    pub fn check_next_item_epoch(&self, epoch: u64) -> LogStoreResult<()> {
        match self {
            TruncateOffset::Chunk {
                epoch: offset_epoch,
                ..
            } => {
                if epoch != *offset_epoch {
                    bail!(
                        "new item epoch {} does not match current chunk offset epoch {}",
                        epoch,
                        offset_epoch
                    );
                }
            }
            TruncateOffset::Barrier {
                epoch: offset_epoch,
            } => {
                if epoch <= *offset_epoch {
                    bail!(
                        "new item epoch {} does not exceed barrier offset epoch {}",
                        epoch,
                        offset_epoch
                    );
                }
            }
        }
        Ok(())
    }
}

#[derive(Debug)]
pub enum LogStoreReadItem {
    StreamChunk {
        chunk: StreamChunk,
        chunk_id: ChunkId,
    },
    Barrier {
        is_checkpoint: bool,
    },
    UpdateVnodeBitmap(Arc<Bitmap>),
}

pub trait LogWriter: Send {
    /// Initialize the log writer with an epoch
    fn init(
        &mut self,
        epoch: EpochPair,
        pause_read_on_bootstrap: bool,
    ) -> impl Future<Output = LogStoreResult<()>> + Send + '_;

    /// Write a stream chunk to the log writer
    fn write_chunk(
        &mut self,
        chunk: StreamChunk,
    ) -> impl Future<Output = LogStoreResult<()>> + Send + '_;

    /// Mark current epoch as finished and sealed, and flush the unconsumed log data.
    fn flush_current_epoch(
        &mut self,
        next_epoch: u64,
        is_checkpoint: bool,
    ) -> impl Future<Output = LogStoreResult<()>> + Send + '_;

    /// Update the vnode bitmap of the log writer
    fn update_vnode_bitmap(
        &mut self,
        new_vnodes: Arc<Bitmap>,
    ) -> impl Future<Output = LogStoreResult<()>> + Send + '_;

    fn pause(&mut self) -> LogStoreResult<()>;

    fn resume(&mut self) -> LogStoreResult<()>;
}

pub trait LogReader: Send + Sized + 'static {
    /// Initialize the log reader. Usually function as waiting for log writer to be initialized.
    fn init(&mut self) -> impl Future<Output = LogStoreResult<()>> + Send + '_;

    /// Emit the next item.
    ///
    /// The implementation should ensure that the future is cancellation safe.
    fn next_item(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(u64, LogStoreReadItem)>> + Send + '_;

    /// Mark that all items emitted so far have been consumed and it is safe to truncate the log
    /// from the current offset.
    fn truncate(&mut self, offset: TruncateOffset) -> LogStoreResult<()>;

    /// Reset the log reader to after the latest truncate offset
    ///
    /// The return flag means whether the log store support rewind
    fn rewind(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(bool, Option<Bitmap>)>> + Send + '_;
}

pub trait LogStoreFactory: Send + 'static {
    type Reader: LogReader;
    type Writer: LogWriter;

    fn build(self) -> impl Future<Output = (Self::Reader, Self::Writer)> + Send;
}

pub struct TransformChunkLogReader<F: Fn(StreamChunk) -> StreamChunk, R: LogReader> {
    f: F,
    inner: R,
}

impl<F: Fn(StreamChunk) -> StreamChunk + Send + 'static, R: LogReader> LogReader
    for TransformChunkLogReader<F, R>
{
    fn init(&mut self) -> impl Future<Output = LogStoreResult<()>> + Send + '_ {
        self.inner.init()
    }

    async fn next_item(&mut self) -> LogStoreResult<(u64, LogStoreReadItem)> {
        let (epoch, item) = self.inner.next_item().await?;
        let item = match item {
            LogStoreReadItem::StreamChunk { chunk, chunk_id } => LogStoreReadItem::StreamChunk {
                chunk: (self.f)(chunk),
                chunk_id,
            },
            other => other,
        };
        Ok((epoch, item))
    }

    fn truncate(&mut self, offset: TruncateOffset) -> LogStoreResult<()> {
        self.inner.truncate(offset)
    }

    fn rewind(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(bool, Option<Bitmap>)>> + Send + '_ {
        self.inner.rewind()
    }
}

pub struct BackpressureMonitoredLogReader<R: LogReader> {
    inner: R,
    /// Start time to wait for new future after poll ready
    wait_new_future_start_time: Option<Instant>,
    wait_new_future_duration_ns: LabelGuardedIntCounter<4>,
}

impl<R: LogReader> BackpressureMonitoredLogReader<R> {
    fn new(inner: R, wait_new_future_duration_ns: LabelGuardedIntCounter<4>) -> Self {
        Self {
            inner,
            wait_new_future_start_time: None,
            wait_new_future_duration_ns,
        }
    }
}

impl<R: LogReader> LogReader for BackpressureMonitoredLogReader<R> {
    fn init(&mut self) -> impl Future<Output = LogStoreResult<()>> + Send + '_ {
        self.wait_new_future_start_time = None;
        self.inner.init()
    }

    fn next_item(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(u64, LogStoreReadItem)>> + Send + '_ {
        if let Some(start_time) = self.wait_new_future_start_time.take() {
            self.wait_new_future_duration_ns
                .inc_by(start_time.elapsed().as_nanos() as _);
        }
        self.inner.next_item().inspect_ok(|_| {
            // Set start time when return ready
            self.wait_new_future_start_time = Some(Instant::now());
        })
    }

    fn truncate(&mut self, offset: TruncateOffset) -> LogStoreResult<()> {
        self.inner.truncate(offset)
    }

    fn rewind(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(bool, Option<Bitmap>)>> + Send + '_ {
        self.inner.rewind().inspect_ok(|_| {
            self.wait_new_future_start_time = None;
        })
    }
}

pub struct MonitoredLogReader<R: LogReader> {
    inner: R,
    read_epoch: u64,
    metrics: LogReaderMetrics,
}

pub struct LogReaderMetrics {
    pub log_store_latest_read_epoch: LabelGuardedIntGauge<4>,
    pub log_store_read_rows: LabelGuardedIntCounter<4>,
    pub log_store_reader_wait_new_future_duration_ns: LabelGuardedIntCounter<4>,
}

impl<R: LogReader> MonitoredLogReader<R> {
    pub fn new(inner: R, metrics: LogReaderMetrics) -> Self {
        Self {
            inner,
            read_epoch: INVALID_EPOCH,
            metrics,
        }
    }
}

impl<R: LogReader> LogReader for MonitoredLogReader<R> {
    async fn init(&mut self) -> LogStoreResult<()> {
        self.inner.init().instrument_await("log_reader_init").await
    }

    async fn next_item(&mut self) -> LogStoreResult<(u64, LogStoreReadItem)> {
        self.inner
            .next_item()
            .instrument_await("log_reader_next_item")
            .await
            .inspect(|(epoch, item)| {
                if self.read_epoch != *epoch {
                    self.read_epoch = *epoch;
                    self.metrics.log_store_latest_read_epoch.set(*epoch as _);
                }
                if let LogStoreReadItem::StreamChunk { chunk, .. } = item {
                    self.metrics
                        .log_store_read_rows
                        .inc_by(chunk.cardinality() as _);
                }
            })
    }

    fn truncate(&mut self, offset: TruncateOffset) -> LogStoreResult<()> {
        self.inner.truncate(offset)
    }

    fn rewind(
        &mut self,
    ) -> impl Future<Output = LogStoreResult<(bool, Option<Bitmap>)>> + Send + '_ {
        self.inner.rewind().instrument_await("log_reader_rewind")
    }
}

#[easy_ext::ext(LogReaderExt)]
impl<T> T
where
    T: LogReader,
{
    pub fn transform_chunk<F: Fn(StreamChunk) -> StreamChunk + Sized>(
        self,
        f: F,
    ) -> TransformChunkLogReader<F, Self> {
        TransformChunkLogReader { f, inner: self }
    }

    pub fn monitored(self, metrics: LogReaderMetrics) -> impl LogReader {
        // TODO: The `clone()` can be avoided if move backpressure inside `MonitoredLogReader`
        let wait_new_future_duration = metrics.log_store_reader_wait_new_future_duration_ns.clone();
        BackpressureMonitoredLogReader::new(
            MonitoredLogReader::new(self, metrics),
            wait_new_future_duration,
        )
    }
}

pub struct MonitoredLogWriter<W: LogWriter> {
    inner: W,
    metrics: LogWriterMetrics,
}

pub struct LogWriterMetrics {
    // Labels: [actor_id, sink_id, sink_name]
    pub log_store_first_write_epoch: LabelGuardedIntGauge<3>,
    pub log_store_latest_write_epoch: LabelGuardedIntGauge<3>,
    pub log_store_write_rows: LabelGuardedIntCounter<3>,
}

impl<W: LogWriter> LogWriter for MonitoredLogWriter<W> {
    async fn init(
        &mut self,
        epoch: EpochPair,
        pause_read_on_bootstrap: bool,
    ) -> LogStoreResult<()> {
        self.metrics
            .log_store_first_write_epoch
            .set(epoch.curr as _);
        self.metrics
            .log_store_latest_write_epoch
            .set(epoch.curr as _);
        self.inner.init(epoch, pause_read_on_bootstrap).await
    }

    async fn write_chunk(&mut self, chunk: StreamChunk) -> LogStoreResult<()> {
        self.metrics
            .log_store_write_rows
            .inc_by(chunk.cardinality() as _);
        self.inner.write_chunk(chunk).await
    }

    async fn flush_current_epoch(
        &mut self,
        next_epoch: u64,
        is_checkpoint: bool,
    ) -> LogStoreResult<()> {
        self.inner
            .flush_current_epoch(next_epoch, is_checkpoint)
            .await?;
        self.metrics
            .log_store_latest_write_epoch
            .set(next_epoch as _);
        Ok(())
    }

    async fn update_vnode_bitmap(&mut self, new_vnodes: Arc<Bitmap>) -> LogStoreResult<()> {
        self.inner.update_vnode_bitmap(new_vnodes).await
    }

    fn pause(&mut self) -> LogStoreResult<()> {
        self.inner.pause()
    }

    fn resume(&mut self) -> LogStoreResult<()> {
        self.inner.resume()
    }
}

#[easy_ext::ext(LogWriterExt)]
impl<T> T
where
    T: LogWriter + Sized,
{
    pub fn monitored(self, metrics: LogWriterMetrics) -> MonitoredLogWriter<T> {
        MonitoredLogWriter {
            inner: self,
            metrics,
        }
    }
}

enum DeliveryFutureManagerItem<F> {
    Chunk {
        chunk_id: ChunkId,
        // earlier future at the front
        futures: VecDeque<F>,
    },
    Barrier,
}

pub struct DeliveryFutureManager<F> {
    future_count: usize,
    max_future_count: usize,
    // earlier items at the front
    items: VecDeque<(u64, DeliveryFutureManagerItem<F>)>,
}

impl<F> DeliveryFutureManager<F> {
    pub fn new(max_future_count: usize) -> Self {
        Self {
            future_count: 0,
            max_future_count,
            items: Default::default(),
        }
    }

    pub fn add_barrier(&mut self, epoch: u64) {
        if let Some((item_epoch, last_item)) = self.items.back() {
            match last_item {
                DeliveryFutureManagerItem::Chunk { .. } => {
                    assert_eq!(*item_epoch, epoch)
                }
                DeliveryFutureManagerItem::Barrier => {
                    assert!(
                        epoch > *item_epoch,
                        "new barrier epoch {} should be greater than prev barrier {}",
                        epoch,
                        item_epoch
                    );
                }
            }
        }
        self.items
            .push_back((epoch, DeliveryFutureManagerItem::Barrier));
    }

    pub fn start_write_chunk(
        &mut self,
        epoch: u64,
        chunk_id: ChunkId,
    ) -> DeliveryFutureManagerAddFuture<'_, F> {
        if let Some((item_epoch, item)) = self.items.back() {
            match item {
                DeliveryFutureManagerItem::Chunk {
                    chunk_id: item_chunk_id,
                    ..
                } => {
                    assert_eq!(epoch, *item_epoch);
                    assert!(
                        chunk_id > *item_chunk_id,
                        "new chunk id {} should be greater than prev chunk id {}",
                        chunk_id,
                        item_chunk_id
                    );
                }
                DeliveryFutureManagerItem::Barrier => {
                    assert!(
                        epoch > *item_epoch,
                        "new chunk epoch {} should be greater than prev barrier: {}",
                        epoch,
                        item_epoch
                    );
                }
            }
        }
        self.items.push_back((
            epoch,
            DeliveryFutureManagerItem::Chunk {
                chunk_id,
                futures: VecDeque::new(),
            },
        ));
        DeliveryFutureManagerAddFuture(self)
    }
}

pub struct DeliveryFutureManagerAddFuture<'a, F>(&'a mut DeliveryFutureManager<F>);

impl<'a, F: TryFuture<Ok = ()> + Unpin + 'static> DeliveryFutureManagerAddFuture<'a, F> {
    /// Add a new future to the latest started written chunk.
    /// The returned bool value indicate whether we have awaited on any previous futures.
    pub async fn add_future_may_await(&mut self, future: F) -> Result<bool, F::Error> {
        let mut has_await = false;
        while self.0.future_count >= self.0.max_future_count {
            self.await_one_delivery().await?;
            has_await = true;
        }
        match self.0.items.back_mut() {
            Some((_, DeliveryFutureManagerItem::Chunk { futures, .. })) => {
                futures.push_back(future);
                self.0.future_count += 1;
                Ok(has_await)
            }
            _ => unreachable!("should add future only after add a new chunk"),
        }
    }

    pub async fn await_one_delivery(&mut self) -> Result<(), F::Error> {
        for (_, item) in &mut self.0.items {
            if let DeliveryFutureManagerItem::Chunk { futures, .. } = item
                && let Some(mut delivery_future) = futures.pop_front()
            {
                self.0.future_count -= 1;
                return poll_fn(|cx| delivery_future.try_poll_unpin(cx)).await;
            } else {
                continue;
            }
        }
        Ok(())
    }

    pub fn future_count(&self) -> usize {
        self.0.future_count
    }

    pub fn max_future_count(&self) -> usize {
        self.0.max_future_count
    }
}

impl<F: TryFuture<Ok = ()> + Unpin + 'static> DeliveryFutureManager<F> {
    pub fn next_truncate_offset(
        &mut self,
    ) -> impl Future<Output = Result<TruncateOffset, F::Error>> + '_ {
        poll_fn(move |cx| {
            let mut latest_offset: Option<TruncateOffset> = None;
            'outer: while let Some((epoch, item)) = self.items.front_mut() {
                match item {
                    DeliveryFutureManagerItem::Chunk { chunk_id, futures } => {
                        while let Some(future) = futures.front_mut() {
                            match future.try_poll_unpin(cx) {
                                Poll::Ready(result) => match result {
                                    Ok(()) => {
                                        self.future_count -= 1;
                                        futures.pop_front();
                                    }
                                    Err(e) => {
                                        return Poll::Ready(Err(e));
                                    }
                                },
                                Poll::Pending => {
                                    break 'outer;
                                }
                            }
                        }

                        // when we reach here, there must not be any pending or error future.
                        // Which means all futures of this stream chunk have been finished
                        assert!(futures.is_empty());
                        latest_offset = Some(TruncateOffset::Chunk {
                            epoch: *epoch,
                            chunk_id: *chunk_id,
                        });
                        self.items.pop_front().expect("items not empty");
                    }
                    DeliveryFutureManagerItem::Barrier => {
                        latest_offset = Some(TruncateOffset::Barrier { epoch: *epoch });
                        self.items.pop_front().expect("items not empty");
                        // Barrier will be yielded anyway
                        break 'outer;
                    }
                }
            }
            if let Some(offset) = latest_offset {
                Poll::Ready(Ok(offset))
            } else {
                Poll::Pending
            }
        })
    }
}

#[cfg(test)]
mod tests {
    use std::future::{poll_fn, Future};
    use std::pin::pin;
    use std::task::Poll;

    use futures::{FutureExt, TryFuture};
    use risingwave_common::util::epoch::test_epoch;
    use tokio::sync::oneshot;
    use tokio::sync::oneshot::Receiver;

    use super::LogStoreResult;
    use crate::sink::log_store::{DeliveryFutureManager, TruncateOffset};

    #[test]
    fn test_truncate_offset_cmp() {
        assert!(
            TruncateOffset::Barrier { epoch: 232 }
                < TruncateOffset::Chunk {
                    epoch: 233,
                    chunk_id: 1
                }
        );
        assert_eq!(
            TruncateOffset::Chunk {
                epoch: 1,
                chunk_id: 1
            },
            TruncateOffset::Chunk {
                epoch: 1,
                chunk_id: 1
            }
        );
        assert!(
            TruncateOffset::Chunk {
                epoch: 1,
                chunk_id: 1
            } < TruncateOffset::Chunk {
                epoch: 1,
                chunk_id: 2
            }
        );
        assert!(
            TruncateOffset::Barrier { epoch: 1 }
                > TruncateOffset::Chunk {
                    epoch: 1,
                    chunk_id: 2
                }
        );
        assert!(
            TruncateOffset::Chunk {
                epoch: 1,
                chunk_id: 2
            } < TruncateOffset::Barrier { epoch: 1 }
        );
        assert!(
            TruncateOffset::Chunk {
                epoch: 2,
                chunk_id: 2
            } > TruncateOffset::Barrier { epoch: 1 }
        );
        assert!(TruncateOffset::Barrier { epoch: 2 } > TruncateOffset::Barrier { epoch: 1 });
    }

    type TestFuture = impl TryFuture<Ok = (), Error = anyhow::Error> + Unpin + 'static;
    fn to_test_future(rx: Receiver<LogStoreResult<()>>) -> TestFuture {
        async move { rx.await.unwrap() }.boxed()
    }

    #[tokio::test]
    async fn test_empty() {
        let mut manager = DeliveryFutureManager::<TestFuture>::new(2);
        let mut future = pin!(manager.next_truncate_offset());
        assert!(poll_fn(|cx| Poll::Ready(future.as_mut().poll(cx)))
            .await
            .is_pending());
    }

    #[tokio::test]
    async fn test_future_delivery_manager_basic() {
        let mut manager = DeliveryFutureManager::new(2);
        let epoch1 = 233;
        let chunk_id1 = 1;
        let (tx1_1, rx1_1) = oneshot::channel();
        let mut write_chunk = manager.start_write_chunk(epoch1, chunk_id1);
        assert!(!write_chunk
            .add_future_may_await(to_test_future(rx1_1))
            .await
            .unwrap());
        assert_eq!(manager.future_count, 1);
        {
            let mut next_truncate_offset = pin!(manager.next_truncate_offset());
            assert!(
                poll_fn(|cx| Poll::Ready(next_truncate_offset.as_mut().poll(cx)))
                    .await
                    .is_pending()
            );
            tx1_1.send(Ok(())).unwrap();
            assert_eq!(
                next_truncate_offset.await.unwrap(),
                TruncateOffset::Chunk {
                    epoch: epoch1,
                    chunk_id: chunk_id1
                }
            );
        }
        assert_eq!(manager.future_count, 0);
        manager.add_barrier(epoch1);
        assert_eq!(
            manager.next_truncate_offset().await.unwrap(),
            TruncateOffset::Barrier { epoch: epoch1 }
        );
    }

    #[tokio::test]
    async fn test_future_delivery_manager_compress_chunk() {
        let mut manager = DeliveryFutureManager::new(10);
        let epoch1 = test_epoch(233);
        let chunk_id1 = 1;
        let chunk_id2 = chunk_id1 + 1;
        let chunk_id3 = chunk_id2 + 1;
        let (tx1_1, rx1_1) = oneshot::channel();
        let (tx1_2, rx1_2) = oneshot::channel();
        let (tx1_3, rx1_3) = oneshot::channel();
        let epoch2 = test_epoch(234);
        let (tx2_1, rx2_1) = oneshot::channel();
        assert!(!manager
            .start_write_chunk(epoch1, chunk_id1)
            .add_future_may_await(to_test_future(rx1_1))
            .await
            .unwrap());
        assert!(!manager
            .start_write_chunk(epoch1, chunk_id2)
            .add_future_may_await(to_test_future(rx1_2))
            .await
            .unwrap());
        assert!(!manager
            .start_write_chunk(epoch1, chunk_id3)
            .add_future_may_await(to_test_future(rx1_3))
            .await
            .unwrap());
        manager.add_barrier(epoch1);
        assert!(!manager
            .start_write_chunk(epoch2, chunk_id1)
            .add_future_may_await(to_test_future(rx2_1))
            .await
            .unwrap());
        assert_eq!(manager.future_count, 4);
        {
            let mut next_truncate_offset = pin!(manager.next_truncate_offset());
            assert!(
                poll_fn(|cx| Poll::Ready(next_truncate_offset.as_mut().poll(cx)))
                    .await
                    .is_pending()
            );
            tx1_2.send(Ok(())).unwrap();
            assert!(
                poll_fn(|cx| Poll::Ready(next_truncate_offset.as_mut().poll(cx)))
                    .await
                    .is_pending()
            );
            tx1_1.send(Ok(())).unwrap();
            // The offset of chunk1 and chunk2 are compressed
            assert_eq!(
                next_truncate_offset.await.unwrap(),
                TruncateOffset::Chunk {
                    epoch: epoch1,
                    chunk_id: chunk_id2
                }
            );
        }
        assert_eq!(manager.future_count, 2);
        {
            let mut next_truncate_offset = pin!(manager.next_truncate_offset());
            assert!(
                poll_fn(|cx| Poll::Ready(next_truncate_offset.as_mut().poll(cx)))
                    .await
                    .is_pending()
            );
            tx1_3.send(Ok(())).unwrap();
            tx2_1.send(Ok(())).unwrap();
            // Emit barrier though later chunk has finished.
            assert_eq!(
                next_truncate_offset.await.unwrap(),
                TruncateOffset::Barrier { epoch: epoch1 }
            );
        }
        assert_eq!(manager.future_count, 1);
        assert_eq!(
            manager.next_truncate_offset().await.unwrap(),
            TruncateOffset::Chunk {
                epoch: epoch2,
                chunk_id: chunk_id1
            }
        );
    }

    #[tokio::test]
    async fn test_future_delivery_manager_await_future() {
        let mut manager = DeliveryFutureManager::new(2);
        let epoch = 233;
        let chunk_id1 = 1;
        let chunk_id2 = chunk_id1 + 1;
        let (tx1_1, rx1_1) = oneshot::channel();
        let (tx1_2, rx1_2) = oneshot::channel();
        let (tx2_1, rx2_1) = oneshot::channel();
        let (tx2_2, rx2_2) = oneshot::channel();

        {
            let mut write_chunk = manager.start_write_chunk(epoch, chunk_id1);
            assert!(!write_chunk
                .add_future_may_await(to_test_future(rx1_1))
                .await
                .unwrap());
            assert!(!write_chunk
                .add_future_may_await(to_test_future(rx1_2))
                .await
                .unwrap());
            assert_eq!(manager.future_count, 2);
        }

        {
            let mut write_chunk = manager.start_write_chunk(epoch, chunk_id2);
            {
                let mut future1 = pin!(write_chunk.add_future_may_await(to_test_future(rx2_1)));
                assert!(poll_fn(|cx| Poll::Ready(future1.as_mut().poll(cx)))
                    .await
                    .is_pending());
                tx1_1.send(Ok(())).unwrap();
                assert!(future1.await.unwrap());
            }
            assert_eq!(2, write_chunk.future_count());
            {
                let mut future2 = pin!(write_chunk.add_future_may_await(to_test_future(rx2_2)));
                assert!(poll_fn(|cx| Poll::Ready(future2.as_mut().poll(cx)))
                    .await
                    .is_pending());
                tx1_2.send(Ok(())).unwrap();
                assert!(future2.await.unwrap());
            }
            assert_eq!(2, write_chunk.future_count());
            {
                let mut future3 = pin!(write_chunk.await_one_delivery());
                assert!(poll_fn(|cx| Poll::Ready(future3.as_mut().poll(cx)))
                    .await
                    .is_pending());
                tx2_1.send(Ok(())).unwrap();
                future3.await.unwrap();
            }
            assert_eq!(1, write_chunk.future_count());
        }

        assert_eq!(
            manager.next_truncate_offset().await.unwrap(),
            TruncateOffset::Chunk {
                epoch,
                chunk_id: chunk_id1
            }
        );

        assert_eq!(1, manager.future_count);

        {
            let mut future = pin!(manager.next_truncate_offset());
            assert!(poll_fn(|cx| Poll::Ready(future.as_mut().poll(cx)))
                .await
                .is_pending());
            tx2_2.send(Ok(())).unwrap();
            assert_eq!(
                future.await.unwrap(),
                TruncateOffset::Chunk {
                    epoch,
                    chunk_id: chunk_id2
                }
            );
        }

        assert_eq!(0, manager.future_count);
    }
}