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
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
//
// Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved.
//
// 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
//

use crate::client_factory::ClientFactoryAsync;
use crate::event::reader_group::ReaderGroupConfigVersioned;
use crate::sync::synchronizer::*;

use pravega_client_shared::{Reader, Scope, ScopedSegment, Segment, SegmentWithRange};

use crate::sync::table::TableError;
#[cfg(test)]
use mockall::automock;
use serde::{Deserialize, Serialize};
use snafu::{ensure, OptionExt, ResultExt, Snafu};
use std::collections::{HashMap, HashSet};
use std::convert::TryInto;
use tracing::{debug, info, warn};

const ASSUMED_LAG_MILLIS: u64 = 30000;
const DEFAULT_INNER_KEY: &str = "default";

const ASSIGNED: &str = "assigned_segments";
const UNASSIGNED: &str = "unassigned_segments";
const FUTURE: &str = "future_segments";
const DISTANCE: &str = "distance_to_tail";

#[derive(Debug, Snafu)]
pub enum ReaderGroupStateError {
    #[snafu(display("Synchronizer error while performing {}: {}", error_msg, source))]
    SyncError {
        error_msg: String,
        source: SynchronizerError,
    },
    ReaderAlreadyOfflineError {
        error_msg: String,
        source: SynchronizerError,
    },
}

impl ReaderGroupStateError {
    pub fn is_precondition(&self) -> bool {
        true
    }
}
/// ReaderGroupState encapsulates all readers states.
pub struct ReaderGroupState {
    /// The sync is a TableSynchronizer that provides API to read or write the internal
    /// reader group state stored on the server side. The internal reader group state contains
    /// the following fields.
    ///
    /// Internal stream that is used to store the ReaderGroupState on the server side.
    /// scoped_synchronizer_stream: ScopedStream,
    ///
    /// Reader group config
    /// config: ReaderGroupConfigVersioned
    ///
    /// This is used to balance the workload among readers in this reader group.
    /// distance_to_tail: HashMap<Reader, u64>
    ///
    /// Maps successor segments to their predecessors. A successor segment will ready to be
    /// read if all its predecessors have been read.
    /// future_segments: HashMap<ScopedSegment, HashSet<i64>>
    ///
    /// Maps active readers to their currently assigned segments.
    /// assigned_segments:  HashMap<Reader, HashMap<ScopedSegment, Offset>>
    ///
    /// Segments waiting to be assigned to readers.
    /// unassigned_segments: HashMap<ScopedSegment, Offset>
    sync: Synchronizer,
}

#[cfg_attr(test, automock)]
impl ReaderGroupState {
    pub(crate) async fn new(
        scope: Scope,
        reader_group_name: String,
        client_factory: &ClientFactoryAsync,
        config: ReaderGroupConfigVersioned,
        segments_to_offsets: HashMap<ScopedSegment, Offset>,
    ) -> ReaderGroupState {
        let mut sync = client_factory
            .create_synchronizer(scope, reader_group_name.clone())
            .await;
        sync.insert(move |table| {
            if table.is_empty() {
                table.insert(
                    "config".to_owned(),
                    DEFAULT_INNER_KEY.to_owned(),
                    "ReaderGroupConfigVersioned".to_owned(),
                    Box::new(config.clone()),
                );
                for (segment, offset) in &segments_to_offsets {
                    table.insert(
                        UNASSIGNED.to_owned(),
                        segment.to_string(),
                        "Offset".to_owned(),
                        Box::new(offset.to_owned()),
                    );
                }
            } else {
                info!("ReaderGroup {:?} already initialized", reader_group_name);
            }
            Ok(())
        })
        .await
        .expect("should initialize table synchronizer");
        ReaderGroupState { sync }
    }

    /// Adds a reader to the reader group state.
    pub async fn add_reader(&mut self, reader: &Reader) -> Result<(), ReaderGroupStateError> {
        info!("Adding reader {:?} to reader group", reader);
        self.sync
            .insert(|table| ReaderGroupState::add_reader_internal(table, reader))
            .await
            .context(SyncError {
                error_msg: format!("add reader {:?}", reader),
            })?;
        Ok(())
    }

    // Internal logic of add_reader method. Separate the actual logic with table synchronizer
    // to facilitate the unit test.
    fn add_reader_internal(table: &mut Update, reader: &Reader) -> Result<(), SynchronizerError> {
        if table.contains_key(ASSIGNED, &reader.to_string()) {
            return Err(SynchronizerError::SyncPreconditionError {
                error_msg: format!("Failed to add online reader {:?}: reader already online", reader),
            });
        }

        // add new reader
        let empty_map: HashMap<ScopedSegment, Offset> = HashMap::new();
        table.insert(
            ASSIGNED.to_owned(),
            reader.to_string(),
            "HashMap<ScopedSegment, Offset>".to_owned(),
            Box::new(empty_map),
        );

        table.insert(
            DISTANCE.to_owned(),
            reader.to_string(),
            "u64".to_owned(),
            Box::new(u64::MAX),
        );
        Ok(())
    }

    pub async fn check_online(&mut self, reader: &Reader) -> bool {
        match self.sync.fetch_updates().await {
            Ok(update_count) => debug!("Number of updates read is {:?}", update_count),
            Err(TableError::TableDoesNotExist { .. }) => return false,
            _ => panic!("Fetch updates failed after all retries"),
        }
        let res = self
            .sync
            .get_inner_map(ASSIGNED)
            .get(&reader.name)
            .map(|v| (v.type_id != TOMBSTONE));
        res.unwrap_or(false)
    }

    /// Returns the active readers in a vector.
    pub async fn get_online_readers(&mut self) -> Vec<Reader> {
        self.sync.fetch_updates().await.expect("should fetch updates");
        ReaderGroupState::get_online_readers_internal(self.sync.get_inner_map(ASSIGNED))
    }

    fn get_online_readers_internal(assigned_segments: HashMap<String, Value>) -> Vec<Reader> {
        assigned_segments
            .keys()
            .map(|k| Reader::from(k.to_owned()))
            .collect::<Vec<Reader>>()
    }

    /// Gets the latest positions for the given reader.
    pub(crate) async fn get_reader_positions(
        &mut self,
        reader: &Reader,
    ) -> Result<HashMap<ScopedSegment, Offset>, ReaderGroupStateError> {
        self.sync.fetch_updates().await.expect("should fetch updates");
        debug!(
            "Assaigned segments {:?} for reader {:?} ",
            self.sync.get_inner_map(ASSIGNED),
            reader
        );
        ReaderGroupState::get_reader_positions_internal(reader, self.sync.get_inner_map(ASSIGNED))
    }

    fn get_reader_positions_internal(
        reader: &Reader,
        assigned_segments: HashMap<String, Value>,
    ) -> Result<HashMap<ScopedSegment, Offset>, ReaderGroupStateError> {
        ReaderGroupState::check_reader_online(&assigned_segments, reader).context(
            ReaderAlreadyOfflineError {
                error_msg: format!("reader {:?} already offline", reader),
            },
        )?;

        Ok(deserialize_from(
            &assigned_segments
                .get(&reader.name)
                .expect("reader must exist")
                .data,
        )
        .expect("Failed to deserialize reader position"))
    }

    /// Updates the latest positions for the given reader.
    pub(crate) async fn update_reader_positions(
        &mut self,
        reader: &Reader,
        latest_positions: HashMap<ScopedSegment, Offset>,
    ) -> Result<(), ReaderGroupStateError> {
        self.sync
            .insert(|table| {
                ReaderGroupState::update_reader_positions_internal(table, reader, &latest_positions)
            })
            .await
            .context(SyncError {
                error_msg: format!("update reader {:?} to position {:?}", reader, latest_positions),
            })
    }

    fn update_reader_positions_internal(
        table: &mut Update,
        reader: &Reader,
        latest_positions: &HashMap<ScopedSegment, Offset>,
    ) -> Result<(), SynchronizerError> {
        let mut owned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;
        if owned_segments.len() != latest_positions.len() {
            warn!(
                "owned segments size {} dose not match latest positions size {}",
                owned_segments.len(),
                latest_positions.len()
            );
        }

        for (segment, offset) in latest_positions {
            owned_segments.entry(segment.to_owned()).and_modify(|v| {
                v.read = offset.read;
            });
        }
        table.insert(
            ASSIGNED.to_owned(),
            reader.to_string(),
            "HashMap<ScopedSegment, Offset>".to_owned(),
            Box::new(owned_segments),
        );
        Ok(())
    }

    /// Removes the given reader from the reader group state and puts segments that are previously
    /// owned by the removed reader to the unassigned list for redistribution.
    pub(crate) async fn remove_reader_default(
        &mut self,
        reader: &Reader,
    ) -> Result<(), ReaderGroupStateError> {
        self.sync
            .insert(|table| ReaderGroupState::remove_reader_internal_default(table, reader))
            .await
            .map_err(|err| match err {
                SynchronizerError::SyncPreconditionError { .. } => {
                    ReaderGroupStateError::ReaderAlreadyOfflineError {
                        error_msg: format!("Reader {:?} already offline", reader),
                        source: err,
                    }
                }
                _ => ReaderGroupStateError::SyncError {
                    error_msg: format!(
                        "remove reader {:?} and put known owned segments to unassigned list",
                        reader
                    ),
                    source: err,
                },
            })?;
        Ok(())
    }

    fn remove_reader_internal_default(table: &mut Update, reader: &Reader) -> Result<(), SynchronizerError> {
        let assigned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;

        for (segment, pos) in assigned_segments {
            table.insert(
                UNASSIGNED.to_owned(),
                segment.to_string(),
                "Offset".to_owned(),
                Box::new(pos),
            );
        }
        table.insert_tombstone(ASSIGNED.to_owned(), reader.to_string())?;
        table.insert_tombstone(DISTANCE.to_owned(), reader.to_string())?;
        Ok(())
    }

    /// Removes the given reader from the reader group state and puts segments that are previously
    /// owned by the removed reader to the unassigned list for redistribution.
    pub(crate) async fn remove_reader(
        &mut self,
        reader: &Reader,
        owned_segments: HashMap<ScopedSegment, Offset>,
    ) -> Result<(), ReaderGroupStateError> {
        self.sync
            .insert(|table| ReaderGroupState::remove_reader_internal(table, reader, &owned_segments))
            .await
            .map_err(|err| match err {
                SynchronizerError::SyncPreconditionError { .. } => {
                    ReaderGroupStateError::ReaderAlreadyOfflineError {
                        error_msg: format!("Reader {:?} already offline", reader),
                        source: err,
                    }
                }
                _ => ReaderGroupStateError::SyncError {
                    error_msg: format!(
                        "remove reader {:?} and put known owned segments to unassigned list",
                        reader
                    ),
                    source: err,
                },
            })?;
        Ok(())
    }

    fn remove_reader_internal(
        table: &mut Update,
        reader: &Reader,
        owned_segments: &HashMap<ScopedSegment, Offset>,
    ) -> Result<(), SynchronizerError> {
        let assigned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;

        for (segment, pos) in assigned_segments {
            // update offset using owned_segments
            let offset = owned_segments.get(&segment).map_or(pos, |v| v.to_owned());

            table.insert(
                UNASSIGNED.to_owned(),
                segment.to_string(),
                "Offset".to_owned(),
                Box::new(offset),
            );
        }
        table.insert_tombstone(ASSIGNED.to_owned(), reader.to_string())?;
        table.insert_tombstone(DISTANCE.to_owned(), reader.to_string())?;
        Ok(())
    }

    /// Compute the number of segments to acquire.
    pub async fn compute_segments_to_acquire_or_release(
        &mut self,
        reader: &Reader,
    ) -> Result<isize, ReaderGroupStateError> {
        match self.sync.fetch_updates().await {
            Ok(update_count) => debug!("Number of updates read is {:?}", update_count),
            Err(TableError::TableDoesNotExist { .. }) => {
                return Err(ReaderGroupStateError::ReaderAlreadyOfflineError {
                    error_msg: String::from("the ReaderGroup is deleted"),
                    source: SynchronizerError::SyncPreconditionError {
                        error_msg: String::from("Precondition failure"),
                    },
                })
            }
            _ => panic!("Fetch updates failed after all retries"),
        }
        let assigned_segment_map = self.sync.get_inner_map(ASSIGNED);
        let num_of_readers = assigned_segment_map.len();
        let mut num_assigned_segments = 0;
        for v in assigned_segment_map.values() {
            let segments: HashMap<ScopedSegment, Offset> =
                deserialize_from(&v.data).expect("deserialize assigned segments");
            num_assigned_segments += segments.len();
        }
        let num_of_segments = num_assigned_segments + self.sync.get_inner_map(UNASSIGNED).len();
        debug!(
            " number of segments {:?}, number of readers {:?} in reader group state",
            num_of_segments, num_of_readers
        );
        let num_segments_per_reader = num_of_segments / num_of_readers;
        let expected_segment_count_per_reader = if num_of_segments % num_of_readers == 0 {
            num_segments_per_reader
        } else {
            num_segments_per_reader + 1
        };
        let current_segment_count = assigned_segment_map.get(&reader.name).map(|v| {
            let seg: HashMap<ScopedSegment, Offset> =
                deserialize_from(&v.data).expect("deserialize of assigned segments");
            seg.len()
        });
        let expected: isize = expected_segment_count_per_reader.try_into().unwrap();
        let current: isize = current_segment_count.unwrap_or_default().try_into().unwrap();
        Ok(expected - current)
    }

    /// Return the list of all segments.
    pub async fn get_segments(&mut self) -> HashSet<ScopedSegment> {
        self.sync.fetch_updates().await.expect("should fetch updates");

        let assigned_segments = self.sync.get_inner_map(ASSIGNED);
        let unassigned_segments = self.sync.get_inner_map(UNASSIGNED);
        println!(
            "Assigned Segments {:?} Unassigned Segment {:?}",
            assigned_segments.len(),
            unassigned_segments.len()
        );
        let mut set: HashSet<ScopedSegment> = HashSet::new();

        for v in assigned_segments.values() {
            let segments: HashMap<ScopedSegment, Offset> =
                deserialize_from(&v.data).expect("deserialize assigned segments");
            set.extend(segments.keys().cloned().collect::<HashSet<ScopedSegment>>())
        }

        set.extend(
            unassigned_segments
                .keys()
                .map(|segment| {
                    let segment_str = &*segment.to_owned();
                    ScopedSegment::from(segment_str)
                })
                .collect::<HashSet<ScopedSegment>>(),
        );
        set
    }

    /// get all assigned and unassigned segments to offset map in ReaderGroup.
    /// This is used to construct StreamCut
    pub async fn get_streamcut(&mut self) -> HashMap<ScopedSegment, Offset> {
        self.sync.fetch_updates().await.expect("should fetch updates");

        let assigned_segments = self.sync.get_inner_map(ASSIGNED);
        let unassigned_segments = self.sync.get_inner_map(UNASSIGNED);
        info!(
            "Assigned Segments {:?} Unassigned Segment {:?}",
            assigned_segments.len(),
            unassigned_segments.len()
        );
        let mut segment_offset_map: HashMap<ScopedSegment, Offset> = HashMap::new();

        for v in assigned_segments.values() {
            let segment_offset: HashMap<ScopedSegment, Offset> =
                deserialize_from(&v.data).expect("deserialize assigned segments");
            segment_offset_map.extend(segment_offset)
        }

        let unassign_segment_offset: HashMap<ScopedSegment, Offset> = unassigned_segments
            .iter()
            .map(|(k, v)| {
                let segment_str = &*k.to_owned();
                (
                    ScopedSegment::from(segment_str),
                    deserialize_from(&v.data).expect("deserialize offset"),
                )
            })
            .collect::<HashMap<ScopedSegment, Offset>>();
        segment_offset_map.extend(unassign_segment_offset);
        debug!("Segment to offset map {:?} from reader group", segment_offset_map);
        segment_offset_map
    }

    /// Assign an unassigned segment to a given reader
    pub async fn assign_segment_to_reader(
        &mut self,
        reader: &Reader,
    ) -> Result<Option<ScopedSegment>, ReaderGroupStateError> {
        let option = self
            .sync
            .insert(|table| ReaderGroupState::assign_segment_to_reader_internal(table, reader))
            .await
            .context(SyncError {
                error_msg: format!("assign segment to reader {:?}", reader),
            })?;

        if let Some(segment_str) = option {
            Ok(Some(ScopedSegment::from(&*segment_str)))
        } else {
            Ok(None)
        }
    }

    fn assign_segment_to_reader_internal(
        table: &mut Update,
        reader: &Reader,
    ) -> Result<Option<String>, SynchronizerError> {
        let mut assigned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;
        let unassigned_segments = ReaderGroupState::get_unassigned_segments_from_table(table);

        if let Some((segment, offset)) = unassigned_segments.into_iter().next() {
            assigned_segments.insert(segment.clone(), offset);
            table.insert(
                ASSIGNED.to_owned(),
                reader.to_string(),
                "HashMap<ScopedSegment, Offset>".to_owned(),
                Box::new(assigned_segments),
            );
            table.insert_tombstone(UNASSIGNED.to_owned(), segment.to_string())?;
            Ok(Some(segment.to_string()))
        } else {
            Ok(None)
        }
    }

    /// Return the list of segments assigned to the requested reader.
    pub async fn get_segments_for_reader(
        &mut self,
        reader: &Reader,
    ) -> Result<HashSet<(ScopedSegment, Offset)>, SynchronizerError> {
        self.sync.fetch_updates().await.expect("should fetch updates");
        let value =
            self.sync
                .get(ASSIGNED, &reader.to_string())
                .ok_or(SynchronizerError::SyncUpdateError {
                    error_msg: format!("reader {} is not online", reader),
                })?;

        let segments: HashMap<ScopedSegment, Offset> =
            deserialize_from(&value.data).expect("deserialize reader owned segments");
        Ok(segments
            .iter()
            .map(|(k, v)| (k.clone(), v.clone()))
            .collect::<HashSet<(ScopedSegment, Offset)>>())
    }

    /// Release a currently assigned segment from the given reader.
    pub async fn release_segment(
        &mut self,
        reader: &Reader,
        segment: &ScopedSegment,
        offset: &Offset,
    ) -> Result<(), ReaderGroupStateError> {
        debug!(
            "Release segment {:?} for reader {:?} at offset {:?}",
            segment, reader, offset.read
        );
        self.sync
            .insert(|table| ReaderGroupState::release_segment_internal(table, reader, segment, offset))
            .await
            .context(SyncError {
                error_msg: format!(
                    "Failed to release segment {:?} with offset {:?} from reader {:?} ",
                    segment, offset, reader
                ),
            })?;
        Ok(())
    }

    /// Find the corresponding segment in the assigned segment list.
    fn release_segment_internal(
        table: &mut Update,
        reader: &Reader,
        segment: &ScopedSegment,
        offset: &Offset,
    ) -> Result<(), SynchronizerError> {
        let mut assigned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;
        let unassigned_segments = ReaderGroupState::get_unassigned_segments_from_table(table);

        let old_offset = assigned_segments.remove(segment).context(SyncUpdateError {
            error_msg: format!(
                "Failed to release segment: should contain only one segment {:?} in assigned list but contains none",
                segment,
            )
        });
        debug!(
            "Removed segment {:?} from assigned segments, the older offset is {:?}",
            segment, old_offset
        );
        // ensure this segment is not part of unassigned segments.
        ensure!(
                !unassigned_segments.contains_key(segment),
                SyncUpdateError {
                    error_msg: format!(
                        "Failed to release segment: unassigned_segment should not have already contained this released segment {:?}",
                        segment
                    )
                });
        // update table
        table.insert(
            ASSIGNED.to_owned(),
            reader.to_string(),
            "HashMap<ScopedSegment, Offset>".to_owned(),
            Box::new(assigned_segments),
        );
        table.insert(
            UNASSIGNED.to_owned(),
            segment.clone().to_string(),
            "Offset".to_owned(),
            Box::new(offset.to_owned()),
        );
        Ok(())
    }

    /// Remove the completed segments and add its successors for next to read.
    /// This should be called by the reader who's reading the current segment. Since a segment
    /// cannot be read by multiple readers, we can assume this won't be called by multiple processors
    /// at the same time.
    pub async fn segment_completed(
        &mut self,
        reader: &Reader,
        segment_completed: &ScopedSegment,
        successors_mapped_to_their_predecessors: &im::HashMap<SegmentWithRange, Vec<Segment>>,
    ) -> Result<(), ReaderGroupStateError> {
        self.sync
            .insert(|table| {
                ReaderGroupState::segment_completed_internal(
                    table,
                    reader,
                    segment_completed,
                    successors_mapped_to_their_predecessors,
                )
            })
            .await
            .context(SyncError {
                error_msg: format!(
                    "segment {:?} assigned to reader {:?} has completed",
                    segment_completed, reader
                ),
            })?;
        Ok(())
    }

    fn segment_completed_internal(
        table: &mut Update,
        reader: &Reader,
        segment_completed: &ScopedSegment,
        successors_mapped_to_their_predecessors: &im::HashMap<SegmentWithRange, Vec<Segment>>,
    ) -> Result<Option<String>, SynchronizerError> {
        let mut assigned_segments = ReaderGroupState::get_reader_owned_segments_from_table(table, reader)?;
        let mut future_segments = ReaderGroupState::get_future_segments_from_table(table);

        // remove completed segment from assigned_segment list
        assigned_segments
            .remove(segment_completed)
            .expect("should have assigned this segment to reader");
        table.insert(
            ASSIGNED.to_owned(),
            reader.to_string(),
            "HashMap<ScopedSegment, Offset>".to_owned(),
            Box::new(assigned_segments),
        );

        // add missing successors to future_segments
        for (segment, list) in successors_mapped_to_their_predecessors {
            if !future_segments.contains_key(&segment.scoped_segment) {
                let required_to_complete: HashSet<_> = list.clone().into_iter().collect();
                table.insert(
                    FUTURE.to_owned(),
                    segment.scoped_segment.to_string(),
                    "HashSet<i64>".to_owned(),
                    Box::new(required_to_complete.clone()),
                );
                // need to update the temp map since later operation may depend on it
                future_segments.insert(segment.scoped_segment.to_owned(), required_to_complete);
            }
        }

        // remove the completed segment from the dependency list
        for (segment, required_to_complete) in &mut future_segments {
            // the hash set needs update
            if required_to_complete.remove(&segment_completed.segment) {
                table.insert(
                    FUTURE.to_owned(),
                    segment.to_string(),
                    "HashSet<i64>".to_owned(),
                    Box::new(required_to_complete.to_owned()),
                );
            }
        }

        // find successors that are ready to read. A successor is ready to read
        // once all its predecessors are completed.
        let ready_to_read = future_segments
            .iter()
            .filter(|&(_segment, set)| set.is_empty())
            .map(|(segment, _set)| segment.to_owned())
            .collect::<Vec<ScopedSegment>>();

        for segment in ready_to_read {
            // add ready to read segments to unassigned_segments
            table.insert(
                UNASSIGNED.to_owned(),
                segment.to_string(),
                "Offset".to_owned(),
                Box::new(Offset::new(0)),
            );
            // remove those from the future_segments
            table.insert_tombstone(FUTURE.to_owned(), segment.to_string())?;
        }
        Ok(None)
    }

    fn get_reader_owned_segments_from_table(
        table: &mut Update,
        reader: &Reader,
    ) -> Result<HashMap<ScopedSegment, Offset>, SynchronizerError> {
        ReaderGroupState::check_reader_online(&table.get_inner_map(ASSIGNED), reader)?;

        let value = table
            .get(ASSIGNED, &reader.to_string())
            .expect("No entries found for reader owned segments");
        let owned_segments: HashMap<ScopedSegment, Offset> =
            deserialize_from(&value.data).expect("deserialize reader owned segments");
        Ok(owned_segments)
    }

    fn get_unassigned_segments_from_table(table: &mut Update) -> HashMap<ScopedSegment, Offset> {
        table
            .get_inner_map(UNASSIGNED)
            .iter()
            .map(|(k, v)| {
                let segment_str = &*k.to_owned();
                (
                    ScopedSegment::from(segment_str),
                    deserialize_from(&v.data).expect("deserialize offset"),
                )
            })
            .collect::<HashMap<ScopedSegment, Offset>>()
    }

    fn get_future_segments_from_table(table: &mut Update) -> HashMap<ScopedSegment, HashSet<Segment>> {
        table
            .get_inner_map(FUTURE)
            .iter()
            .map(|(k, v)| {
                let segment_str = &*k.to_owned();
                (
                    ScopedSegment::from(segment_str),
                    deserialize_from(&v.data).expect("deserialize hashset"),
                )
            })
            .collect::<HashMap<ScopedSegment, HashSet<Segment>>>()
    }

    fn check_reader_online(
        assigned: &HashMap<String, Value>,
        reader: &Reader,
    ) -> Result<(), SynchronizerError> {
        if assigned.contains_key(&reader.name) {
            Ok(())
        } else {
            Err(SynchronizerError::SyncPreconditionError {
                error_msg: format!("reader {} is not online", reader.name),
            })
        }
    }
}

#[derive(new, Serialize, Deserialize, PartialEq, Debug, Clone, Hash, Eq)]
pub struct Offset {
    /// The client has read to this offset and handle the result to the application/caller.
    /// But some events before this offset may not have been processed by application/caller.
    /// In case of failure, those unprocessed events may need to be read from application/caller
    /// again.
    pub read: i64,
}

#[cfg(test)]
mod test {
    use super::*;
    use crate::sync::synchronizer::{serialize, Value};
    use lazy_static::*;
    use ordered_float::OrderedFloat;
    use pravega_client_shared::{Scope, Segment, Stream};

    lazy_static! {
        static ref READER: Reader = Reader::from("test".to_owned());
        static ref SEGMENT: ScopedSegment = ScopedSegment {
            scope: Scope {
                name: "scope".to_string(),
            },
            stream: Stream {
                name: "scope".to_string(),
            },
            segment: Segment {
                number: 0,
                tx_id: None,
            },
        };
        static ref SEGMENT_TEST: ScopedSegment = SEGMENT.clone();
    }

    fn set_up() -> Update {
        let offset = Box::new(Offset::new(0));
        let data = serialize(&*offset).expect("serialize value");

        let mut unassigned_segments = HashMap::new();
        unassigned_segments.insert(
            SEGMENT_TEST.to_string(),
            Value {
                type_id: "Offset".to_owned(),
                data,
            },
        );

        let mut map = HashMap::new();
        map.insert(UNASSIGNED.to_owned(), unassigned_segments);

        let counter = HashMap::new();
        let table = Update::new(map, counter, vec![], vec![]);

        assert!(table.contains_outer_key(UNASSIGNED));
        assert!(table.contains_key(UNASSIGNED, &SEGMENT_TEST.to_string()));

        table
    }

    #[test]
    fn test_reader_group_state() {
        // set up
        let mut table = set_up();

        // add a reader
        ReaderGroupState::add_reader_internal(&mut table, &READER).expect("add reader");

        assert!(
            table.contains_outer_key(ASSIGNED),
            "assigned_segments outer map should be created automatically"
        );
        assert!(
            table.contains_key(ASSIGNED, &READER.to_string()),
            "should contain inner key"
        );

        // get online readers
        let mut readers = ReaderGroupState::get_online_readers_internal(table.get_inner_map(ASSIGNED));

        assert_eq!(
            readers.pop().expect("get reader"),
            READER.clone(),
            "should have online reader added"
        );
        assert!(readers.is_empty(), "should have only one reader");

        // assign a segment to a reader
        ReaderGroupState::assign_segment_to_reader_internal(&mut table, &READER)
            .expect("assign segment to reader");
        let segments =
            ReaderGroupState::get_reader_positions_internal(&READER, table.get_inner_map(ASSIGNED))
                .expect("get reader positions");

        assert_eq!(segments.len(), 1, "should have assigned one segment the reader");
        assert_eq!(
            segments.get(&SEGMENT_TEST).expect("get segment"),
            &Offset { read: 0 },
            "added segment should be as expected"
        );

        // update reader position
        let new_offset = Offset { read: 10 };
        let mut update = HashMap::new();
        update.insert(SEGMENT_TEST.clone(), new_offset.clone());

        ReaderGroupState::update_reader_positions_internal(&mut table, &READER, &update)
            .expect("update reader position");
        let segments =
            ReaderGroupState::get_reader_positions_internal(&READER, table.get_inner_map(ASSIGNED))
                .expect("get reader positions");
        assert_eq!(segments.len(), 1, "reader should contain one owned segment");
        assert_eq!(
            segments.get(&SEGMENT_TEST).expect("get segment"),
            &new_offset,
            "the offset of owned segment should be updated"
        );

        // segment completed
        let mut successor0 = SEGMENT_TEST.clone();
        successor0.segment.number = 1;

        let successor0_range = SegmentWithRange {
            scoped_segment: successor0.clone(),
            min_key: OrderedFloat(0.0),
            max_key: OrderedFloat(0.5),
        };

        let mut successor1 = SEGMENT_TEST.clone();
        successor1.segment.number = 2;
        let successor1_range = SegmentWithRange {
            scoped_segment: successor1.clone(),
            min_key: OrderedFloat(0.5),
            max_key: OrderedFloat(1.0),
        };

        let mut successors_mapped_to_their_predecessors = im::HashMap::new();
        successors_mapped_to_their_predecessors.insert(
            successor0_range,
            vec![Segment {
                number: 0,
                tx_id: None,
            }],
        );
        successors_mapped_to_their_predecessors.insert(
            successor1_range,
            vec![Segment {
                number: 0,
                tx_id: None,
            }],
        );

        ReaderGroupState::segment_completed_internal(
            &mut table,
            &READER,
            &SEGMENT_TEST,
            &successors_mapped_to_their_predecessors,
        )
        .expect("reader segment completed");
        assert!(table.contains_key(UNASSIGNED, &successor0.to_string()));
        assert!(table.contains_key(UNASSIGNED, &successor1.to_string()));

        ReaderGroupState::assign_segment_to_reader_internal(&mut table, &READER)
            .expect("assign segment to reader");
        ReaderGroupState::assign_segment_to_reader_internal(&mut table, &READER)
            .expect("assign segment to reader");

        // release segment from reader
        let new_offset = Offset { read: 10 };
        ReaderGroupState::release_segment_internal(&mut table, &READER, &successor0, &new_offset)
            .expect("release segment");

        let segments =
            ReaderGroupState::get_reader_positions_internal(&READER, table.get_inner_map(ASSIGNED))
                .expect("get reader positions");
        assert_eq!(
            segments.len(),
            1,
            "reader should contain 1 segment since the other one is released"
        );

        // reader offline
        let mut segments = HashMap::new();
        segments.insert(successor0.clone(), Offset { read: 0 });
        segments.insert(successor1.clone(), Offset { read: 0 });

        ReaderGroupState::remove_reader_internal(&mut table, &READER, &segments)
            .expect("remove online reader");
        let res = ReaderGroupState::remove_reader_internal(&mut table, &READER, &segments)
            .expect_err("Excepted error");
        match res {
            SynchronizerError::SyncPreconditionError { .. } => {}
            _ => assert!(false, "Invalid error returned"),
        }
        assert_eq!(table.get_inner_map(UNASSIGNED).len(), 2);
    }
}