loro_internal/
undo.rs

1use std::{cell::RefCell, collections::VecDeque, sync::Arc};
2
3use crate::sync::{AtomicU64, Mutex};
4use either::Either;
5use loro_common::{
6    ContainerID, Counter, CounterSpan, HasIdSpan, IdSpan, LoroError, LoroResult, LoroValue, PeerID,
7};
8use parking_lot::lock_api::ReentrantMutex;
9use rustc_hash::{FxHashMap, FxHashSet};
10use tracing::{debug_span, info_span, instrument};
11
12use crate::{
13    change::{get_sys_timestamp, Timestamp},
14    cursor::{AbsolutePosition, Cursor},
15    delta::TreeExternalDiff,
16    event::{Diff, EventTriggerKind},
17    version::Frontiers,
18    ContainerDiff, DiffEvent, DocDiff, LoroDoc, Subscription,
19};
20
21/// A batch of diffs.
22///
23/// You can use `loroDoc.apply_diff(diff)` to apply the diff to the document.
24#[derive(Debug, Clone, Default)]
25pub struct DiffBatch {
26    pub cid_to_events: FxHashMap<ContainerID, Diff>,
27    pub order: Vec<ContainerID>,
28}
29
30impl DiffBatch {
31    pub fn new(diff: Vec<DocDiff>) -> Self {
32        let mut map: FxHashMap<ContainerID, Diff> = Default::default();
33        let mut order: Vec<ContainerID> = Vec::with_capacity(diff.len());
34        for d in diff.into_iter() {
35            for item in d.diff.into_iter() {
36                let old = map.insert(item.id.clone(), item.diff);
37                assert!(old.is_none());
38                order.push(item.id.clone());
39            }
40        }
41
42        Self {
43            cid_to_events: map,
44            order,
45        }
46    }
47
48    pub fn compose(&mut self, other: &Self) {
49        if other.cid_to_events.is_empty() {
50            return;
51        }
52
53        for (id, diff) in other.iter() {
54            if let Some(this_diff) = self.cid_to_events.get_mut(id) {
55                this_diff.compose_ref(diff);
56            } else {
57                self.cid_to_events.insert(id.clone(), diff.clone());
58                self.order.push(id.clone());
59            }
60        }
61    }
62
63    pub fn transform(&mut self, other: &Self, left_priority: bool) {
64        if other.cid_to_events.is_empty() || self.cid_to_events.is_empty() {
65            return;
66        }
67
68        for (idx, diff) in self.cid_to_events.iter_mut() {
69            if let Some(b_diff) = other.cid_to_events.get(idx) {
70                diff.transform(b_diff, left_priority);
71            }
72        }
73    }
74
75    pub fn clear(&mut self) {
76        self.cid_to_events.clear();
77        self.order.clear();
78    }
79
80    pub fn iter(&self) -> impl Iterator<Item = (&ContainerID, &Diff)> + '_ {
81        self.order
82            .iter()
83            .map(|cid| (cid, self.cid_to_events.get(cid).unwrap()))
84    }
85
86    #[allow(clippy::should_implement_trait)]
87    pub fn into_iter(self) -> impl Iterator<Item = (ContainerID, Diff)> {
88        let mut cid_to_events = self.cid_to_events;
89        self.order.into_iter().map(move |cid| {
90            let d = cid_to_events.remove(&cid).unwrap();
91            (cid, d)
92        })
93    }
94}
95
96fn transform_cursor(
97    cursor_with_pos: &mut CursorWithPos,
98    remote_diff: &DiffBatch,
99    doc: &LoroDoc,
100    container_remap: &FxHashMap<ContainerID, ContainerID>,
101) {
102    let mut container_changed = false;
103    let mut cid = &cursor_with_pos.cursor.container;
104    while let Some(new_cid) = container_remap.get(cid) {
105        cid = new_cid;
106        container_changed = true;
107    }
108
109    if cursor_with_pos.cursor.id.is_none() {
110        // We don't need to transform a cursor that always points to the leftmost or rightmost position
111        if container_changed {
112            cursor_with_pos.cursor.container = cid.clone();
113        }
114        return;
115    }
116
117    if let Some(diff) = remote_diff.cid_to_events.get(cid) {
118        let new_pos = diff.transform_cursor(cursor_with_pos.pos.pos, false);
119        cursor_with_pos.pos.pos = new_pos;
120    };
121
122    let new_pos = cursor_with_pos.pos.pos;
123    match doc.get_handler(cid.clone()).unwrap() {
124        crate::handler::Handler::Text(h) => {
125            let Some(new_cursor) = h.get_cursor_internal(new_pos, cursor_with_pos.pos.side, false)
126            else {
127                return;
128            };
129
130            cursor_with_pos.cursor = new_cursor;
131        }
132        crate::handler::Handler::List(h) => {
133            let Some(new_cursor) = h.get_cursor(new_pos, cursor_with_pos.pos.side) else {
134                return;
135            };
136
137            cursor_with_pos.cursor = new_cursor;
138        }
139        crate::handler::Handler::MovableList(h) => {
140            let Some(new_cursor) = h.get_cursor(new_pos, cursor_with_pos.pos.side) else {
141                return;
142            };
143
144            cursor_with_pos.cursor = new_cursor;
145        }
146        crate::handler::Handler::Map(_) => {}
147        crate::handler::Handler::Tree(_) => {}
148        crate::handler::Handler::Unknown(_) => {}
149        #[cfg(feature = "counter")]
150        crate::handler::Handler::Counter(_) => {}
151    }
152}
153
154/// UndoManager is responsible for managing undo/redo from the current peer's perspective.
155///
156/// Undo/local is local: it cannot be used to undone the changes made by other peers.
157/// If you want to undo changes made by other peers, you may need to use the time travel feature.
158///
159/// PeerID cannot be changed during the lifetime of the UndoManager
160pub struct UndoManager {
161    peer: Arc<AtomicU64>,
162    container_remap: Arc<Mutex<FxHashMap<ContainerID, ContainerID>>>,
163    inner: Arc<parking_lot::ReentrantMutex<RefCell<UndoManagerInner>>>,
164    _peer_id_change_sub: Subscription,
165    _undo_sub: Subscription,
166    doc: LoroDoc,
167}
168
169impl std::fmt::Debug for UndoManager {
170    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
171        f.debug_struct("UndoManager")
172            .field("peer", &self.peer)
173            .field("container_remap", &self.container_remap)
174            .field("inner", &self.inner)
175            .finish()
176    }
177}
178
179#[derive(Debug, Clone, Copy, PartialEq, Eq)]
180pub enum UndoOrRedo {
181    Undo,
182    Redo,
183}
184
185impl UndoOrRedo {
186    fn opposite(&self) -> UndoOrRedo {
187        match self {
188            Self::Undo => Self::Redo,
189            Self::Redo => Self::Undo,
190        }
191    }
192}
193
194/// When a undo/redo item is pushed, the undo manager will call the on_push callback to get the meta data of the undo item.
195/// The returned cursors will be recorded for a new pushed undo item.
196pub type OnPush = Box<
197    dyn for<'a> Fn(UndoOrRedo, CounterSpan, Option<DiffEvent<'a>>) -> UndoItemMeta + Send + Sync,
198>;
199pub type OnPop = Box<dyn Fn(UndoOrRedo, CounterSpan, UndoItemMeta) + Send + Sync>;
200
201struct UndoManagerInner {
202    next_counter: Option<Counter>,
203    undo_stack: Stack,
204    redo_stack: Stack,
205    processing_undo: bool,
206    last_undo_time: i64,
207    merge_interval_in_ms: i64,
208    max_stack_size: usize,
209    exclude_origin_prefixes: Vec<Box<str>>,
210    last_popped_selection: Option<Vec<CursorWithPos>>,
211    on_push: Option<OnPush>,
212    on_pop: Option<OnPop>,
213    group: Option<UndoGroup>,
214}
215
216impl std::fmt::Debug for UndoManagerInner {
217    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
218        f.debug_struct("UndoManagerInner")
219            .field("latest_counter", &self.next_counter)
220            .field("undo_stack", &self.undo_stack)
221            .field("redo_stack", &self.redo_stack)
222            .field("processing_undo", &self.processing_undo)
223            .field("last_undo_time", &self.last_undo_time)
224            .field("merge_interval", &self.merge_interval_in_ms)
225            .field("max_stack_size", &self.max_stack_size)
226            .field("exclude_origin_prefixes", &self.exclude_origin_prefixes)
227            .field("group", &self.group)
228            .finish()
229    }
230}
231
232#[derive(Debug, Clone, Default)]
233struct UndoGroup {
234    start_counter: Counter,
235    affected_cids: FxHashSet<ContainerID>,
236}
237
238impl UndoGroup {
239    pub fn new(start_counter: Counter) -> Self {
240        Self {
241            start_counter,
242            affected_cids: Default::default(),
243        }
244    }
245}
246
247#[derive(Debug)]
248struct Stack {
249    stack: VecDeque<(VecDeque<StackItem>, Arc<Mutex<DiffBatch>>)>,
250    size: usize,
251}
252
253#[derive(Debug, Clone)]
254struct StackItem {
255    span: CounterSpan,
256    meta: UndoItemMeta,
257}
258
259/// The metadata of an undo item.
260///
261/// The cursors inside the metadata will be transformed by remote operations as well.
262/// So that when the item is popped, users can restore the cursors position correctly.
263#[derive(Debug, Default, Clone)]
264pub struct UndoItemMeta {
265    pub value: LoroValue,
266    pub cursors: Vec<CursorWithPos>,
267}
268
269#[derive(Debug, Clone)]
270pub struct CursorWithPos {
271    pub cursor: Cursor,
272    pub pos: AbsolutePosition,
273}
274
275impl UndoItemMeta {
276    pub fn new() -> Self {
277        Self {
278            value: LoroValue::Null,
279            cursors: Default::default(),
280        }
281    }
282
283    /// It's assumed that the cursor is just acquired before the ops that
284    /// need to be undo/redo.
285    ///
286    /// We need to rely on the validity of the original pos value
287    pub fn add_cursor(&mut self, cursor: &Cursor) {
288        self.cursors.push(CursorWithPos {
289            cursor: cursor.clone(),
290            pos: AbsolutePosition {
291                pos: cursor.origin_pos,
292                side: cursor.side,
293            },
294        });
295    }
296
297    pub fn set_value(&mut self, value: LoroValue) {
298        self.value = value;
299    }
300}
301
302impl Stack {
303    pub fn new() -> Self {
304        let mut stack = VecDeque::new();
305        stack.push_back((VecDeque::new(), Arc::new(Mutex::new(Default::default()))));
306        Stack { stack, size: 0 }
307    }
308
309    /// Peek the top-most StackItem's metadata without modifying the stack.
310    ///
311    /// Returns None if the stack is empty.
312    fn peek_top_meta(&self) -> Option<UndoItemMeta> {
313        if self.is_empty() {
314            return None;
315        }
316
317        for (items, _) in self.stack.iter().rev() {
318            if let Some(item) = items.back() {
319                return Some(item.meta.clone());
320            }
321        }
322
323        None
324    }
325
326    pub fn pop(&mut self) -> Option<(StackItem, Arc<Mutex<DiffBatch>>)> {
327        while self.stack.back().unwrap().0.is_empty() && self.stack.len() > 1 {
328            let (_, diff) = self.stack.pop_back().unwrap();
329            let diff = diff.lock().unwrap();
330            if !diff.cid_to_events.is_empty() {
331                self.stack
332                    .back_mut()
333                    .unwrap()
334                    .1
335                    .lock()
336                    .unwrap()
337                    .compose(&diff);
338            }
339        }
340
341        if self.stack.len() == 1 && self.stack.back().unwrap().0.is_empty() {
342            // If the stack is empty, we need to clear the remote diff
343            self.stack.back_mut().unwrap().1.lock().unwrap().clear();
344            return None;
345        }
346
347        self.size -= 1;
348        let last = self.stack.back_mut().unwrap();
349        last.0.pop_back().map(|x| (x, last.1.clone()))
350        // If this row in stack is empty, we don't pop it right away
351        // Because we still need the remote diff to be available.
352        // Cursor position transformation relies on the remote diff in the same row.
353    }
354
355    pub fn push(&mut self, span: CounterSpan, meta: UndoItemMeta) {
356        self.push_with_merge(span, meta, false, None)
357    }
358
359    pub fn push_with_merge(
360        &mut self,
361        span: CounterSpan,
362        meta: UndoItemMeta,
363        can_merge: bool,
364        group: Option<&UndoGroup>,
365    ) {
366        let last = self.stack.back_mut().unwrap();
367        let last_remote_diff = last.1.lock().unwrap();
368
369        // Check if the remote diff is disjoint with the current undo group
370        let is_disjoint_group = group.is_some_and(|g| {
371            g.affected_cids.iter().all(|cid| {
372                last_remote_diff
373                    .cid_to_events
374                    .get(cid)
375                    .is_none_or(|diff| diff.is_empty())
376            })
377        });
378
379        // Can't merge if remote diffs exist and it's not disjoint with the current undo group
380        let should_create_new_entry =
381            !last_remote_diff.cid_to_events.is_empty() && !is_disjoint_group;
382
383        if should_create_new_entry {
384            // Create a new entry in the stack
385            drop(last_remote_diff);
386            let mut v = VecDeque::new();
387            v.push_back(StackItem { span, meta });
388            self.stack
389                .push_back((v, Arc::new(Mutex::new(DiffBatch::default()))));
390            self.size += 1;
391            return;
392        }
393
394        // Try to merge with the previous entry if allowed
395        if can_merge {
396            if let Some(last_span) = last.0.back_mut() {
397                if last_span.span.end == span.start {
398                    // Merge spans by extending the end of the last span
399                    last_span.span.end = span.end;
400                    return;
401                }
402            }
403        }
404
405        // Add as a new item to the existing entry
406        self.size += 1;
407        last.0.push_back(StackItem { span, meta });
408    }
409
410    pub fn compose_remote_event(&mut self, diff: &[&ContainerDiff]) {
411        if self.is_empty() {
412            return;
413        }
414
415        let remote_diff = &mut self.stack.back_mut().unwrap().1;
416        let mut remote_diff = remote_diff.lock().unwrap();
417        for e in diff {
418            if let Some(d) = remote_diff.cid_to_events.get_mut(&e.id) {
419                d.compose_ref(&e.diff);
420            } else {
421                remote_diff
422                    .cid_to_events
423                    .insert(e.id.clone(), e.diff.clone());
424                remote_diff.order.push(e.id.clone());
425            }
426        }
427    }
428
429    pub fn transform_based_on_this_delta(&mut self, diff: &DiffBatch) {
430        if self.is_empty() {
431            return;
432        }
433        let remote_diff = &mut self.stack.back_mut().unwrap().1;
434        remote_diff.lock().unwrap().transform(diff, false);
435    }
436
437    pub fn clear(&mut self) {
438        self.stack = VecDeque::new();
439        self.stack.push_back((VecDeque::new(), Default::default()));
440        self.size = 0;
441    }
442
443    pub fn is_empty(&self) -> bool {
444        self.size == 0
445    }
446
447    pub fn len(&self) -> usize {
448        self.size
449    }
450
451    fn pop_front(&mut self) {
452        if self.is_empty() {
453            return;
454        }
455
456        self.size -= 1;
457        let first = self.stack.front_mut().unwrap();
458        let f = first.0.pop_front();
459        assert!(f.is_some());
460        if first.0.is_empty() {
461            self.stack.pop_front();
462        }
463    }
464
465    fn set_top_meta(&mut self, meta: UndoItemMeta) {
466        let Some(top) = self.stack.back_mut() else {
467            return;
468        };
469        let Some(last) = top.0.back_mut() else {
470            return;
471        };
472        last.meta = meta;
473    }
474}
475
476impl Default for Stack {
477    fn default() -> Self {
478        Stack::new()
479    }
480}
481
482impl UndoManagerInner {
483    fn new(last_counter: Counter) -> Self {
484        Self {
485            next_counter: Some(last_counter),
486            undo_stack: Default::default(),
487            redo_stack: Default::default(),
488            processing_undo: false,
489            merge_interval_in_ms: 0,
490            last_undo_time: 0,
491            max_stack_size: usize::MAX,
492            exclude_origin_prefixes: vec![],
493            last_popped_selection: None,
494            on_pop: None,
495            on_push: None,
496            group: None,
497        }
498    }
499
500    /// Returns true if a given container diff is disjoint with the current group.
501    /// They are disjoint if they have no overlap in changed container ids.
502    fn is_disjoint_with_group(&self, diff: &[&ContainerDiff]) -> bool {
503        let Some(group) = &self.group else {
504            return false;
505        };
506
507        diff.iter().all(|d| !group.affected_cids.contains(&d.id))
508    }
509
510    fn record_checkpoint(this: &RefCell<Self>, latest_counter: Counter, event: Option<DiffEvent>) {
511        let previous_counter = this.borrow().next_counter;
512
513        if Some(latest_counter) == this.borrow().next_counter {
514            return;
515        }
516
517        if this.borrow().next_counter.is_none() {
518            this.borrow_mut().next_counter = Some(latest_counter);
519            return;
520        }
521
522        if let Some(group) = &mut this.borrow_mut().group {
523            event.iter().for_each(|e| {
524                e.events.iter().for_each(|e| {
525                    group.affected_cids.insert(e.id.clone());
526                })
527            });
528        }
529
530        let now = get_sys_timestamp() as Timestamp;
531        let span = CounterSpan::new(this.borrow().next_counter.unwrap(), latest_counter);
532        let meta = this
533            .borrow()
534            .on_push
535            .as_ref()
536            .map(|x| x(UndoOrRedo::Undo, span, event))
537            .unwrap_or_default();
538
539        let mut this = this.borrow_mut();
540        let this: &mut Self = &mut this;
541        // Wether the change is within the accepted merge interval
542        let in_merge_interval = now - this.last_undo_time < this.merge_interval_in_ms;
543
544        // If group is active, but there is nothing in the group, don't merge
545        // If the group is active and it's not the first push in the group, merge
546        let group_should_merge = this.group.is_some()
547            && match (
548                previous_counter,
549                this.group.as_ref().map(|g| g.start_counter),
550            ) {
551                (Some(previous), Some(active)) => previous != active,
552                _ => true,
553            };
554
555        let should_merge = !this.undo_stack.is_empty() && (in_merge_interval || group_should_merge);
556
557        if should_merge {
558            this.undo_stack
559                .push_with_merge(span, meta, true, this.group.as_ref());
560        } else {
561            this.last_undo_time = now;
562            this.undo_stack.push(span, meta);
563        }
564
565        this.next_counter = Some(latest_counter);
566        this.redo_stack.clear();
567        while this.undo_stack.len() > this.max_stack_size {
568            this.undo_stack.pop_front();
569        }
570    }
571}
572
573fn get_counter_end(doc: &LoroDoc, peer: PeerID) -> Counter {
574    doc.oplog()
575        .lock()
576        .unwrap()
577        .vv()
578        .get(&peer)
579        .cloned()
580        .unwrap_or(0)
581}
582
583impl UndoManager {
584    pub fn new(doc: &LoroDoc) -> Self {
585        let peer = Arc::new(AtomicU64::new(doc.peer_id()));
586        let peer_clone = peer.clone();
587        let peer_clone2 = peer.clone();
588        let inner = Arc::new(ReentrantMutex::new(RefCell::new(UndoManagerInner::new(
589            get_counter_end(doc, doc.peer_id()),
590        ))));
591        let inner_clone = inner.clone();
592        let inner_clone2 = inner.clone();
593        let remap_containers = Arc::new(Mutex::new(FxHashMap::default()));
594        let remap_containers_clone = remap_containers.clone();
595        let undo_sub = doc.subscribe_root(Arc::new(move |event| match event.event_meta.by {
596            EventTriggerKind::Local => {
597                // TODO: PERF undo can be significantly faster if we can get
598                // the DiffBatch for undo here
599                let lock = inner_clone.lock();
600                if lock.borrow().processing_undo {
601                    return;
602                }
603                if let Some(id) = event
604                    .event_meta
605                    .to
606                    .iter()
607                    .find(|x| x.peer == peer_clone.load(std::sync::atomic::Ordering::Relaxed))
608                {
609                    let should_exclude = lock
610                        .borrow()
611                        .exclude_origin_prefixes
612                        .iter()
613                        .any(|x| event.event_meta.origin.starts_with(&**x));
614                    if should_exclude {
615                        // If the event is from the excluded origin, we don't record it
616                        // in the undo stack. But we need to record its effect like it's
617                        // a remote event.
618                        let mut inner = lock.borrow_mut();
619                        inner.undo_stack.compose_remote_event(event.events);
620                        inner.redo_stack.compose_remote_event(event.events);
621                        inner.next_counter = Some(id.counter + 1);
622                    } else {
623                        UndoManagerInner::record_checkpoint(&lock, id.counter + 1, Some(event));
624                    }
625                }
626            }
627            EventTriggerKind::Import => {
628                let lock = inner_clone.lock();
629                let mut inner = lock.borrow_mut();
630
631                for e in event.events {
632                    if let Diff::Tree(tree) = &e.diff {
633                        for item in &tree.diff {
634                            let target = item.target;
635                            if let TreeExternalDiff::Create { .. } = &item.action {
636                                // If the concurrent event is a create event, it may bring the deleted tree node back,
637                                // so we need to remove it from the remap of the container.
638                                remap_containers_clone
639                                    .lock()
640                                    .unwrap()
641                                    .remove(&target.associated_meta_container());
642                            }
643                        }
644                    }
645                }
646
647                let is_import_disjoint = inner.is_disjoint_with_group(event.events);
648
649                inner.undo_stack.compose_remote_event(event.events);
650                inner.redo_stack.compose_remote_event(event.events);
651
652                // If the import is not disjoint, we end the active group
653                // all subsequent changes will be new undo items
654                if !is_import_disjoint {
655                    inner.group = None;
656                }
657            }
658            EventTriggerKind::Checkout => {
659                let lock = inner_clone.lock();
660                let mut inner = lock.borrow_mut();
661                inner.undo_stack.clear();
662                inner.redo_stack.clear();
663                inner.next_counter = None;
664            }
665        }));
666
667        let sub = doc.subscribe_peer_id_change(Box::new(move |id| {
668            let lock = inner_clone2.lock();
669            let mut inner = lock.borrow_mut();
670            inner.undo_stack.clear();
671            inner.redo_stack.clear();
672            inner.next_counter = Some(id.counter);
673            peer_clone2.store(id.peer, std::sync::atomic::Ordering::Relaxed);
674            true
675        }));
676
677        UndoManager {
678            peer,
679            container_remap: remap_containers,
680            inner,
681            _peer_id_change_sub: sub,
682            _undo_sub: undo_sub,
683            doc: doc.clone(),
684        }
685    }
686
687    pub fn group_start(&self) -> LoroResult<()> {
688        let lock = self.inner.lock();
689        let mut inner = lock.borrow_mut();
690
691        if inner.group.is_some() {
692            return Err(LoroError::UndoGroupAlreadyStarted);
693        }
694
695        inner.group = Some(UndoGroup::new(inner.next_counter.unwrap()));
696
697        Ok(())
698    }
699
700    pub fn group_end(&self) {
701        self.inner.lock().borrow_mut().group = None;
702    }
703
704    pub fn peer(&self) -> PeerID {
705        self.peer.load(std::sync::atomic::Ordering::Relaxed)
706    }
707
708    pub fn set_merge_interval(&self, interval: i64) {
709        self.inner.lock().borrow_mut().merge_interval_in_ms = interval;
710    }
711
712    pub fn set_max_undo_steps(&self, size: usize) {
713        self.inner.lock().borrow_mut().max_stack_size = size;
714    }
715
716    pub fn add_exclude_origin_prefix(&self, prefix: &str) {
717        self.inner
718            .lock()
719            .borrow_mut()
720            .exclude_origin_prefixes
721            .push(prefix.into());
722    }
723
724    pub fn record_new_checkpoint(&self) -> LoroResult<()> {
725        // Use implicit-style barrier to preserve next-commit options across
726        // an empty commit before undo/redo processing.
727        self.doc.with_barrier(|| {});
728        let counter = get_counter_end(&self.doc, self.peer());
729        UndoManagerInner::record_checkpoint(&self.inner.lock(), counter, None);
730        Ok(())
731    }
732
733    #[instrument(skip_all)]
734    pub fn undo(&self) -> LoroResult<bool> {
735        self.perform(
736            |x| &mut x.undo_stack,
737            |x| &mut x.redo_stack,
738            UndoOrRedo::Undo,
739        )
740    }
741
742    #[instrument(skip_all)]
743    pub fn redo(&self) -> LoroResult<bool> {
744        self.perform(
745            |x| &mut x.redo_stack,
746            |x| &mut x.undo_stack,
747            UndoOrRedo::Redo,
748        )
749    }
750
751    fn perform(
752        &self,
753        get_stack: impl Fn(&mut UndoManagerInner) -> &mut Stack,
754        get_opposite: impl Fn(&mut UndoManagerInner) -> &mut Stack,
755        kind: UndoOrRedo,
756    ) -> LoroResult<bool> {
757        let doc = &self.doc.clone();
758        // When in the undo/redo loop, the new undo/redo stack item should restore the selection
759        // to the state it was in before the item that was popped two steps ago from the stack.
760        //
761        //                          ┌────────────┐
762        //                          │Selection 1 │
763        //                          └─────┬──────┘
764        //                                │   Some
765        //                                ▼   ops
766        //                          ┌────────────┐
767        //                          │Selection 2 │
768        //                          └─────┬──────┘
769        //                                │   Some
770        //                                ▼   ops
771        //                          ┌────────────┐
772        //                          │Selection 3 │◁ ─ ─ ─ ─ ─ ─ ─  Restore  ─ ─ ─
773        //                          └─────┬──────┘                               │
774        //                                │
775        //                                │                                      │
776        //                                │                              ┌ ─ ─ ─ ─ ─ ─ ─
777        //           Enter the            │   Undo ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─▶   Push Redo   │
778        //           undo/redo ─ ─ ─ ▶    ▼                              └ ─ ─ ─ ─ ─ ─ ─
779        //             loop         ┌────────────┐                               │
780        //                          │Selection 2 │◁─ ─ ─  Restore  ─
781        //                          └─────┬──────┘                  │            │
782        //                                │
783        //                                │                         │            │
784        //                                │                 ┌ ─ ─ ─ ─ ─ ─ ─
785        //                                │   Undo ─ ─ ─ ─ ▶   Push Redo   │     │
786        //                                ▼                 └ ─ ─ ─ ─ ─ ─ ─
787        //                          ┌────────────┐                  │            │
788        //                          │Selection 1 │
789        //                          └─────┬──────┘                  │            │
790        //                                │   Redo ◀ ─ ─ ─ ─ ─ ─ ─ ─
791        //                                ▼                                      │
792        //                          ┌────────────┐
793        //         ┌   Restore   ─ ▷│Selection 2 │                               │
794        //                          └─────┬──────┘
795        //         │                      │                                      │
796        // ┌ ─ ─ ─ ─ ─ ─ ─                │
797        //    Push Undo   │◀─ ─ ─ ─ ─ ─ ─ │   Redo ◀ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
798        // └ ─ ─ ─ ─ ─ ─ ─                ▼
799        //         │                ┌────────────┐
800        //                          │Selection 3 │
801        //         │                └─────┬──────┘
802        //          ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ▶ │   Undo
803        //                                ▼
804        //                          ┌────────────┐
805        //                          │Selection 2 │
806        //                          └────────────┘
807        //
808        // Because users may change the selections during the undo/redo loop, it's
809        // more stable to keep the selection stored in the last stack item
810        // rather than using the current selection directly.
811        self.record_new_checkpoint()?;
812        let end_counter = get_counter_end(doc, self.peer());
813        let mut top = {
814            let lock = self.inner.lock();
815            let mut inner = lock.borrow_mut();
816            inner.processing_undo = true;
817            get_stack(&mut inner).pop()
818        };
819
820        let mut executed = false;
821        while let Some((mut span, remote_diff)) = top {
822            let mut next_push_selection = None;
823            {
824                let inner = self.inner.clone();
825                // We need to clone this because otherwise <transform_delta> will be applied to the same remote diff
826                let remote_change_clone = remote_diff.lock().unwrap().clone();
827                let commit = doc.undo_internal(
828                    IdSpan {
829                        peer: self.peer(),
830                        counter: span.span,
831                    },
832                    &mut self.container_remap.lock().unwrap(),
833                    Some(&remote_change_clone),
834                    &mut |diff| {
835                        info_span!("transform remote diff").in_scope(|| {
836                            let inner = inner.lock();
837                            // <transform_delta>
838                            get_stack(&mut inner.borrow_mut()).transform_based_on_this_delta(diff);
839                        });
840                    },
841                )?;
842                drop(commit);
843                let inner = self.inner.lock();
844                let mut is_some = false;
845
846                if let Some(on_pop) = inner.borrow().on_pop.as_ref() {
847                    is_some = true;
848                    for cursor in span.meta.cursors.iter_mut() {
849                        // <cursor_transform> We need to transform cursor here.
850                        // Note that right now <transform_delta> is already done,
851                        // remote_diff is also transformed by it now (that's what we need).
852                        transform_cursor(
853                            cursor,
854                            &remote_diff.lock().unwrap(),
855                            doc,
856                            &self.container_remap.lock().unwrap(),
857                        );
858                    }
859
860                    on_pop(kind, span.span, span.meta.clone());
861                }
862                if is_some {
863                    let take = inner.borrow_mut().last_popped_selection.take();
864                    next_push_selection = take;
865                    inner.borrow_mut().last_popped_selection = Some(span.meta.cursors);
866                }
867            }
868            let new_counter = get_counter_end(doc, self.peer());
869            if end_counter != new_counter {
870                let inner = self.inner.lock();
871                let mut meta = inner
872                    .borrow()
873                    .on_push
874                    .as_ref()
875                    .map(|x| {
876                        x(
877                            kind.opposite(),
878                            CounterSpan::new(end_counter, new_counter),
879                            None,
880                        )
881                    })
882                    .unwrap_or_default();
883
884                if matches!(kind, UndoOrRedo::Undo)
885                    && get_opposite(&mut inner.borrow_mut()).is_empty()
886                {
887                    // If it's the first undo, we use the cursors from the users
888                } else if let Some(inner) = next_push_selection.take() {
889                    // Otherwise, we use the cursors from the undo/redo loop
890                    meta.cursors = inner;
891                }
892
893                get_opposite(&mut inner.borrow_mut())
894                    .push(CounterSpan::new(end_counter, new_counter), meta);
895                inner.borrow_mut().next_counter = Some(new_counter);
896                executed = true;
897                break;
898            } else {
899                // continue to pop the undo item as this undo is a no-op
900                top = get_stack(&mut self.inner.lock().borrow_mut()).pop();
901                continue;
902            }
903        }
904
905        self.inner.lock().borrow_mut().processing_undo = false;
906        Ok(executed)
907    }
908
909    pub fn can_undo(&self) -> bool {
910        !self.inner.lock().borrow().undo_stack.is_empty()
911    }
912
913    pub fn can_redo(&self) -> bool {
914        !self.inner.lock().borrow().redo_stack.is_empty()
915    }
916
917    pub fn undo_count(&self) -> usize {
918        self.inner.lock().borrow().undo_stack.len()
919    }
920
921    pub fn redo_count(&self) -> usize {
922        self.inner.lock().borrow().redo_stack.len()
923    }
924
925    /// Get the metadata of the top undo stack item, if any.
926    pub fn top_undo_meta(&self) -> Option<UndoItemMeta> {
927        self.inner.lock().borrow().undo_stack.peek_top_meta()
928    }
929
930    /// Get the metadata of the top redo stack item, if any.
931    pub fn top_redo_meta(&self) -> Option<UndoItemMeta> {
932        self.inner.lock().borrow().redo_stack.peek_top_meta()
933    }
934
935    /// Get the value associated with the top undo stack item, if any.
936    pub fn top_undo_value(&self) -> Option<LoroValue> {
937        self.top_undo_meta().map(|m| m.value)
938    }
939
940    /// Get the value associated with the top redo stack item, if any.
941    pub fn top_redo_value(&self) -> Option<LoroValue> {
942        self.top_redo_meta().map(|m| m.value)
943    }
944
945    pub fn set_on_push(&self, on_push: Option<OnPush>) {
946        self.inner.lock().borrow_mut().on_push = on_push;
947    }
948
949    pub fn set_on_pop(&self, on_pop: Option<OnPop>) {
950        self.inner.lock().borrow_mut().on_pop = on_pop;
951    }
952
953    pub fn clear(&self) {
954        self.inner.lock().borrow_mut().undo_stack.clear();
955        self.inner.lock().borrow_mut().redo_stack.clear();
956    }
957
958    pub fn set_top_undo_meta(&self, meta: UndoItemMeta) {
959        self.inner.lock().borrow_mut().undo_stack.set_top_meta(meta);
960    }
961
962    pub fn set_top_redo_meta(&self, meta: UndoItemMeta) {
963        self.inner.lock().borrow_mut().redo_stack.set_top_meta(meta);
964    }
965}
966
967/// Undo the given spans of operations.
968///
969/// # Parameters
970///
971/// - `spans`: A vector of tuples where each tuple contains an `IdSpan` and its associated `Frontiers`.
972///   - `IdSpan`: Represents a span of operations identified by an ID.
973///   - `Frontiers`: Represents the deps of the given id_span
974/// - `latest_frontiers`: The latest frontiers of the document
975/// - `calc_diff`: A closure that takes two `Frontiers` and calculates the difference between them, returning a `DiffBatch`.
976///
977/// # Returns
978///
979/// - `DiffBatch`: Applying this batch on the `latest_frontiers` will undo the ops in the given spans.
980pub(crate) fn undo(
981    spans: Vec<(IdSpan, Frontiers)>,
982    last_frontiers_or_last_bi: Either<&Frontiers, &DiffBatch>,
983    calc_diff: impl Fn(&Frontiers, &Frontiers) -> DiffBatch,
984    on_last_event_a: &mut dyn FnMut(&DiffBatch),
985) -> DiffBatch {
986    // The process of performing undo is:
987    //
988    // 0. Split the span into a series of continuous spans. There is no external dep within each continuous span.
989    //
990    // For each continuous span_i:
991    //
992    // 1. a. Calculate the event of checkout from id_span.last to id_span.deps, call it Ai. It undo the ops in the current span.
993    //    b. Calculate A'i = Ai + T(Ci-1, Ai) if i > 0, otherwise A'i = Ai.
994    //       NOTE: A'i can undo the ops in the current span and the previous spans, if it's applied on the id_span.last version.
995    // 2. Calculate the event of checkout from id_span.last to [the next span's last id] or [the latest version], call it Bi.
996    // 3. Transform event A'i based on Bi, call it Ci
997    // 4. If span_i is the last span, apply Ci to the current state.
998
999    // -------------------------------------------------------
1000    // 0. Split the span into a series of continuous spans
1001    // -------------------------------------------------------
1002
1003    let mut last_ci: Option<DiffBatch> = None;
1004    for i in 0..spans.len() {
1005        debug_span!("Undo", ?i, "Undo span {:?}", &spans[i]).in_scope(|| {
1006            let (this_id_span, this_deps) = &spans[i];
1007            // ---------------------------------------
1008            // 1.a Calc event A_i
1009            // ---------------------------------------
1010            let mut event_a_i = debug_span!("1. Calc event A_i").in_scope(|| {
1011                // Checkout to the last id of the id_span
1012                calc_diff(&this_id_span.id_last().into(), this_deps)
1013            });
1014
1015            // println!("event_a_i: {:?}", event_a_i);
1016
1017            // ---------------------------------------
1018            // 2. Calc event B_i
1019            // ---------------------------------------
1020            let stack_diff_batch;
1021            let event_b_i = 'block: {
1022                let next = if i + 1 < spans.len() {
1023                    spans[i + 1].0.id_last().into()
1024                } else {
1025                    match last_frontiers_or_last_bi {
1026                        Either::Left(last_frontiers) => last_frontiers.clone(),
1027                        Either::Right(right) => break 'block right,
1028                    }
1029                };
1030                stack_diff_batch = Some(calc_diff(&this_id_span.id_last().into(), &next));
1031                stack_diff_batch.as_ref().unwrap()
1032            };
1033
1034            // println!("event_b_i: {:?}", event_b_i);
1035
1036            // event_a_prime can undo the ops in the current span and the previous spans
1037            let mut event_a_prime = if let Some(mut last_ci) = last_ci.take() {
1038                // ------------------------------------------------------------------------------
1039                // 1.b Transform and apply Ci-1 based on Ai, call it A'i
1040                // ------------------------------------------------------------------------------
1041                last_ci.transform(&event_a_i, true);
1042
1043                event_a_i.compose(&last_ci);
1044                event_a_i
1045            } else {
1046                event_a_i
1047            };
1048            if i == spans.len() - 1 {
1049                on_last_event_a(&event_a_prime);
1050            }
1051            // --------------------------------------------------
1052            // 3. Transform event A'_i based on B_i, call it C_i
1053            // --------------------------------------------------
1054            event_a_prime.transform(event_b_i, true);
1055
1056            // println!("event_a_prime: {:?}", event_a_prime);
1057
1058            let c_i = event_a_prime;
1059            last_ci = Some(c_i);
1060        });
1061    }
1062
1063    last_ci.unwrap()
1064}