loro_internal/
diff_calc.rs

1use std::{num::NonZeroU16, sync::Arc};
2
3#[cfg(feature = "counter")]
4mod counter;
5#[cfg(feature = "counter")]
6pub(crate) use counter::CounterDiffCalculator;
7pub(super) mod tree;
8mod unknown;
9use either::Either;
10use generic_btree::rle::HasLength as _;
11use itertools::Itertools;
12
13use enum_dispatch::enum_dispatch;
14use rustc_hash::{FxHashMap, FxHashSet};
15use loro_common::{
16    CompactIdLp, ContainerID, Counter, HasCounterSpan, IdFull, IdLp, IdSpan, LoroValue, PeerID, ID,
17};
18use loro_delta::DeltaRope;
19use smallvec::SmallVec;
20use tracing::{info_span, instrument};
21
22use crate::{
23    change::Lamport,
24    container::{
25        idx::ContainerIdx,
26        list::list_op::InnerListOp,
27        richtext::{
28            richtext_state::{RichtextStateChunk, TextChunk},
29            AnchorType, CrdtRopeDelta, RichtextChunk, RichtextChunkValue, RichtextTracker, StyleOp,
30        },
31    },
32    cursor::AbsolutePosition,
33    delta::{
34        Delta, DeltaItem, DeltaValue, ElementDelta, MapDelta, MapValue, MovableListInnerDelta,
35    },
36    event::{DiffVariant, InternalDiff},
37    op::{InnerContent, RichOp, SliceRange, SliceWithId},
38    span::{HasId, HasLamport},
39    version::Frontiers,
40    InternalString, VersionVector,
41};
42
43use self::tree::TreeDiffCalculator;
44
45use self::unknown::UnknownDiffCalculator;
46
47use super::{event::InternalContainerDiff, oplog::OpLog};
48
49/// Calculate the diff between two versions. given [OpLog][super::oplog::OpLog]
50/// and [AppState][super::state::AppState].
51///
52/// TODO: persist diffCalculator and skip processed version
53#[derive(Debug)]
54pub struct DiffCalculator {
55    /// ContainerIdx -> (depth, calculator)
56    ///
57    /// if depth is None, we need to calculate it again
58    calculators: FxHashMap<ContainerIdx, (Option<NonZeroU16>, ContainerDiffCalculator)>,
59    retain_mode: DiffCalculatorRetainMode,
60}
61
62#[derive(Debug)]
63enum DiffCalculatorRetainMode {
64    /// The diff calculator can only be used once.
65    Once { used: bool },
66    /// The diff calculator will be persisted and can be reused after the diff calc is done.
67    Persist,
68}
69
70/// This mode defines how the diff is calculated and how it should be applied on the state.
71#[derive(Debug, Clone, PartialEq, Eq, Copy)]
72pub(crate) enum DiffMode {
73    /// This is the most general mode of diff calculation.
74    ///
75    /// When applying `Checkout` diff, we already know the current state of the affected registers.
76    /// So there is no need to compare the lamport values.
77    ///
78    /// It can be used whenever a user want to switch to a different version.
79    /// But it is also the slowest mode. It relies on the `ContainerHistoryCache`, which is expensive to build and maintain in memory.
80    Checkout,
81    /// This mode is used when the user imports new updates.
82    ///
83    /// When applying `Import` diff, we may need to know the the current state.
84    /// For example, we may need to compare the current register's lamport with the update's lamport to decide
85    /// what's the new value.
86    ///
87    /// It has stricter requirements than `Checkout`:
88    ///
89    /// - The target version vector must be greater than the current version vector.
90    Import,
91    /// This mode is used when the user imports new updates and all the updates are guaranteed to greater than the current version.
92    ///
93    /// It has stricter requirements than `Import`.
94    /// - All the updates are greater than the current version. No update is concurrent to the current version.
95    /// - So LCA is always the `from` version
96    ImportGreaterUpdates,
97    /// This mode is used when we don't need to build CRDTs to calculate the difference. It is the fastest mode.
98    ///
99    /// It has stricter requirements than `ImportGreaterUpdates`.
100    /// - In `ImportGreaterUpdates`, all the updates are guaranteed to be greater than the current version.
101    /// - In `Linear`, all the updates are ordered, no concurrent update exists.
102    Linear,
103}
104
105#[derive(Debug, Clone, Copy)]
106pub(crate) struct DiffCalcVersionInfo<'a> {
107    from_vv: &'a VersionVector,
108    to_vv: &'a VersionVector,
109    from_frontiers: &'a Frontiers,
110    to_frontiers: &'a Frontiers,
111}
112
113impl DiffCalculator {
114    /// Create a new diff calculator.
115    ///
116    /// If `persist` is true, the diff calculator will be persisted after the diff calc is done.
117    /// This is useful when we need to cache the diff calculator for future use. But it is slower
118    /// for importing updates and requires more memory.
119    pub fn new(persist: bool) -> Self {
120        Self {
121            calculators: Default::default(),
122            retain_mode: if persist {
123                DiffCalculatorRetainMode::Persist
124            } else {
125                DiffCalculatorRetainMode::Once { used: false }
126            },
127        }
128    }
129
130    #[allow(unused)]
131    pub(crate) fn get_calc(&self, container: ContainerIdx) -> Option<&ContainerDiffCalculator> {
132        self.calculators.get(&container).map(|(_, c)| c)
133    }
134
135    /// Calculate the diff between two versions.
136    ///
137    /// Return the diff and the origin diff mode (it's not the diff mode used by the diff calculator.
138    /// It's the expected diff mode inferred from the two version, which can reflect the direction of the
139    /// change).
140    pub(crate) fn calc_diff_internal(
141        &mut self,
142        oplog: &super::oplog::OpLog,
143        before: &crate::VersionVector,
144        before_frontiers: &Frontiers,
145        after: &crate::VersionVector,
146        after_frontiers: &Frontiers,
147        container_filter: Option<&dyn Fn(ContainerIdx) -> bool>,
148    ) -> (Vec<InternalContainerDiff>, DiffMode) {
149        if before == after {
150            return (Vec::new(), DiffMode::Linear);
151        }
152
153        let s = tracing::span!(tracing::Level::INFO, "DiffCalc", ?before, ?after,);
154        let _e = s.enter();
155
156        let mut merged = before.clone();
157        merged.merge(after);
158        let (lca, origin_diff_mode, iter) =
159            oplog.iter_from_lca_causally(before, before_frontiers, after, after_frontiers);
160        let mut diff_mode = origin_diff_mode;
161        match &mut self.retain_mode {
162            DiffCalculatorRetainMode::Once { used } => {
163                if *used {
164                    panic!("DiffCalculator with retain_mode Once can only be used once");
165                }
166            }
167            DiffCalculatorRetainMode::Persist => {
168                diff_mode = DiffMode::Checkout;
169            }
170        }
171
172        let affected_set = {
173            loro_common::debug!("LCA: {:?} mode={:?}", &lca, diff_mode);
174            let mut started_set = FxHashSet::default();
175            for (change, (start_counter, end_counter), vv) in iter {
176                let iter_start = change
177                    .ops
178                    .binary_search_by(|op| op.ctr_last().cmp(&start_counter))
179                    .unwrap_or_else(|e| e);
180                let mut visited = FxHashSet::default();
181                for mut op in &change.ops.vec()[iter_start..] {
182                    if op.counter >= end_counter {
183                        break;
184                    }
185
186                    let idx = op.container;
187                    if let Some(filter) = container_filter {
188                        if !filter(idx) {
189                            continue;
190                        }
191                    }
192
193                    // slice the op if needed
194                    // PERF: we can skip the slice by using the RichOp::new_slice
195                    let stack_sliced_op;
196                    if op.ctr_last() < start_counter {
197                        continue;
198                    }
199
200                    if op.counter < start_counter || op.ctr_end() > end_counter {
201                        stack_sliced_op = Some(op.slice(
202                            (start_counter as usize).saturating_sub(op.counter as usize),
203                            op.atom_len().min((end_counter - op.counter) as usize),
204                        ));
205                        op = stack_sliced_op.as_ref().unwrap();
206                    }
207
208                    let vv = &mut vv.borrow_mut();
209                    vv.extend_to_include_end_id(ID::new(change.peer(), op.counter));
210                    let container = op.container;
211                    let depth = oplog.arena.get_depth(container);
212                    let (old_depth, calculator) = self.get_or_create_calc(container, depth);
213                    // checkout use the same diff_calculator, the depth of calculator is not updated
214                    // That may cause the container to be considered deleted
215                    if *old_depth != depth {
216                        *old_depth = depth;
217                    }
218
219                    if !started_set.contains(&op.container) {
220                        started_set.insert(container);
221                        calculator.start_tracking(oplog, &lca, diff_mode);
222                    }
223
224                    if visited.contains(&op.container) {
225                        // don't checkout if we have already checked out this container in this round
226                        calculator.apply_change(oplog, RichOp::new_by_change(&change, op), None);
227                    } else {
228                        calculator.apply_change(
229                            oplog,
230                            RichOp::new_by_change(&change, op),
231                            Some(vv),
232                        );
233                        visited.insert(container);
234                    }
235                }
236            }
237
238            Some(started_set)
239        };
240
241        // Because we need to get correct `bring_back` value that indicates container is created during this round of diff calc,
242        // we need to iterate from parents to children. i.e. from smaller depth to larger depth.
243        let mut new_containers = FxHashSet::default();
244        let mut container_id_to_depth = FxHashMap::default();
245        let mut all: Vec<(Option<NonZeroU16>, ContainerIdx)> = if let Some(set) = affected_set {
246            // only visit the affected containers
247            set.into_iter()
248                .map(|x| {
249                    let (depth, _) = self.calculators.get_mut(&x).unwrap();
250                    (*depth, x)
251                })
252                .collect()
253        } else {
254            self.calculators
255                .iter_mut()
256                .map(|(x, (depth, _))| (*depth, *x))
257                .collect()
258        };
259        let mut ans = FxHashMap::default();
260        let info = DiffCalcVersionInfo {
261            from_vv: before,
262            to_vv: after,
263            from_frontiers: before_frontiers,
264            to_frontiers: after_frontiers,
265        };
266        while !all.is_empty() {
267            // sort by depth and lamport, ensure we iterate from top to bottom
268            all.sort_by_key(|x| x.0);
269            for (_, container_idx) in std::mem::take(&mut all) {
270                if ans.contains_key(&container_idx) {
271                    continue;
272                }
273                let (depth, calc) = self.calculators.get_mut(&container_idx).unwrap();
274                if depth.is_none() {
275                    let d = oplog.arena.get_depth(container_idx);
276                    if d != *depth {
277                        *depth = d;
278                        all.push((*depth, container_idx));
279                        continue;
280                    }
281                }
282                let id = oplog.arena.idx_to_id(container_idx).unwrap();
283                let bring_back = new_containers.remove(&id);
284
285                info_span!("CalcDiff", ?id).in_scope(|| {
286                    let (diff, diff_mode) = calc.calculate_diff(container_idx, oplog, info, |c| {
287                        new_containers.insert(c.clone());
288                        container_id_to_depth
289                            .insert(c.clone(), depth.and_then(|d| d.checked_add(1)));
290                        oplog.arena.register_container(c);
291                    });
292                    calc.finish_this_round();
293                    if !diff.is_empty() || bring_back {
294                        ans.insert(
295                            container_idx,
296                            (
297                                *depth,
298                                InternalContainerDiff {
299                                    idx: container_idx,
300                                    bring_back,
301                                    diff: diff.into(),
302                                    diff_mode,
303                                },
304                            ),
305                        );
306                    }
307                });
308            }
309        }
310
311        while !new_containers.is_empty() {
312            for id in std::mem::take(&mut new_containers) {
313                // Registration can be lazy; ensure it is registered so we can proceed
314                let idx = oplog.arena.register_container(&id);
315                if ans.contains_key(&idx) {
316                    continue;
317                }
318                let depth = container_id_to_depth.remove(&id).unwrap();
319                ans.insert(
320                    idx,
321                    (
322                        depth,
323                        InternalContainerDiff {
324                            idx,
325                            bring_back: true,
326                            diff: DiffVariant::None,
327                            diff_mode: DiffMode::Checkout,
328                        },
329                    ),
330                );
331            }
332        }
333
334        (
335            ans.into_values().map(|x| x.1).collect_vec(),
336            origin_diff_mode,
337        )
338    }
339
340    // TODO: we may remove depth info
341    pub(crate) fn get_or_create_calc(
342        &mut self,
343        idx: ContainerIdx,
344        depth: Option<NonZeroU16>,
345    ) -> &mut (Option<NonZeroU16>, ContainerDiffCalculator) {
346        self.calculators
347            .entry(idx)
348            .or_insert_with(|| match idx.get_type() {
349                crate::ContainerType::Text => (
350                    depth,
351                    ContainerDiffCalculator::Richtext(RichtextDiffCalculator::new()),
352                ),
353                crate::ContainerType::Map => (
354                    depth,
355                    ContainerDiffCalculator::Map(MapDiffCalculator::new(idx)),
356                ),
357                crate::ContainerType::List => (
358                    depth,
359                    ContainerDiffCalculator::List(ListDiffCalculator::default()),
360                ),
361                crate::ContainerType::Tree => (
362                    depth,
363                    ContainerDiffCalculator::Tree(TreeDiffCalculator::new(idx)),
364                ),
365                crate::ContainerType::Unknown(_) => (
366                    depth,
367                    ContainerDiffCalculator::Unknown(unknown::UnknownDiffCalculator),
368                ),
369                crate::ContainerType::MovableList => (
370                    depth,
371                    ContainerDiffCalculator::MovableList(MovableListDiffCalculator::new(idx)),
372                ),
373                #[cfg(feature = "counter")]
374                crate::ContainerType::Counter => (
375                    depth,
376                    ContainerDiffCalculator::Counter(CounterDiffCalculator::new(idx)),
377                ),
378            })
379    }
380}
381
382/// DiffCalculator should track the history first before it can calculate the difference.
383///
384/// So we need it to first apply all the ops between the two versions.
385///
386/// NOTE: not every op between two versions are included in a certain container.
387/// So there may be some ops that cannot be seen by the container.
388///
389#[enum_dispatch]
390pub(crate) trait DiffCalculatorTrait {
391    fn start_tracking(&mut self, oplog: &OpLog, vv: &crate::VersionVector, mode: DiffMode);
392    fn apply_change(
393        &mut self,
394        oplog: &OpLog,
395        op: crate::op::RichOp,
396        vv: Option<&crate::VersionVector>,
397    );
398    fn calculate_diff(
399        &mut self,
400        idx: ContainerIdx,
401        oplog: &OpLog,
402        info: DiffCalcVersionInfo,
403        on_new_container: impl FnMut(&ContainerID),
404    ) -> (InternalDiff, DiffMode);
405    /// This round of diff calc is finished, we can clear the cache
406    fn finish_this_round(&mut self);
407}
408
409#[enum_dispatch(DiffCalculatorTrait)]
410#[derive(Debug)]
411pub(crate) enum ContainerDiffCalculator {
412    Map(MapDiffCalculator),
413    List(ListDiffCalculator),
414    Richtext(RichtextDiffCalculator),
415    Tree(TreeDiffCalculator),
416    MovableList(MovableListDiffCalculator),
417    #[cfg(feature = "counter")]
418    Counter(counter::CounterDiffCalculator),
419    Unknown(UnknownDiffCalculator),
420}
421
422#[derive(Debug)]
423pub(crate) struct MapDiffCalculator {
424    container_idx: ContainerIdx,
425    changed: FxHashMap<InternalString, Option<MapValue>>,
426    current_mode: DiffMode,
427}
428
429impl MapDiffCalculator {
430    pub(crate) fn new(container_idx: ContainerIdx) -> Self {
431        Self {
432            container_idx,
433            changed: Default::default(),
434            current_mode: DiffMode::Checkout,
435        }
436    }
437}
438
439impl DiffCalculatorTrait for MapDiffCalculator {
440    fn start_tracking(
441        &mut self,
442        _oplog: &crate::OpLog,
443        _vv: &crate::VersionVector,
444        mode: DiffMode,
445    ) {
446        self.changed.clear();
447        self.current_mode = mode;
448    }
449
450    fn apply_change(
451        &mut self,
452        _oplog: &crate::OpLog,
453        op: crate::op::RichOp,
454        _vv: Option<&crate::VersionVector>,
455    ) {
456        if matches!(self.current_mode, DiffMode::Checkout) {
457            // We need to use history cache anyway
458            return;
459        }
460
461        let map = op.raw_op().content.as_map().unwrap();
462        let new_value = MapValue {
463            value: map.value.clone(),
464            peer: op.peer,
465            lamp: op.lamport(),
466        };
467        match self.changed.get(&map.key) {
468            Some(Some(old_value)) if old_value > &new_value => {}
469            _ => {
470                self.changed.insert(map.key.clone(), Some(new_value));
471            }
472        }
473    }
474
475    fn finish_this_round(&mut self) {
476        self.changed.clear();
477        self.current_mode = DiffMode::Checkout;
478    }
479
480    fn calculate_diff(
481        &mut self,
482        _idx: ContainerIdx,
483        oplog: &super::oplog::OpLog,
484        DiffCalcVersionInfo { from_vv, to_vv, .. }: DiffCalcVersionInfo,
485        mut on_new_container: impl FnMut(&ContainerID),
486    ) -> (InternalDiff, DiffMode) {
487        match self.current_mode {
488            DiffMode::Checkout | DiffMode::Import => oplog.with_history_cache(|h| {
489                let checkout_index = &h.get_checkout_index().map;
490                let mut changed = Vec::new();
491                let from_map = checkout_index.get_container_latest_op_at_vv(
492                    self.container_idx,
493                    from_vv,
494                    Lamport::MAX,
495                    oplog,
496                );
497                let mut to_map = checkout_index.get_container_latest_op_at_vv(
498                    self.container_idx,
499                    to_vv,
500                    Lamport::MAX,
501                    oplog,
502                );
503
504                for (k, peek_from) in from_map.iter() {
505                    let peek_to = to_map.remove(k);
506                    match peek_to {
507                        None => changed.push((k.clone(), None)),
508                        Some(b) => {
509                            if peek_from.value != b.value {
510                                changed.push((k.clone(), Some(b)))
511                            }
512                        }
513                    }
514                }
515
516                for (k, peek_to) in to_map.into_iter() {
517                    changed.push((k, Some(peek_to)));
518                }
519
520                let mut updated =
521                    FxHashMap::with_capacity_and_hasher(changed.len(), Default::default());
522                for (key, value) in changed {
523                    let value = value.map(|v| {
524                        let value = v.value.clone();
525                        if let Some(LoroValue::Container(c)) = &value {
526                            on_new_container(c);
527                        }
528
529                        MapValue {
530                            value,
531                            lamp: v.lamport,
532                            peer: v.peer,
533                        }
534                    });
535
536                    updated.insert(key, value);
537                }
538
539                (InternalDiff::Map(MapDelta { updated }), DiffMode::Checkout)
540            }),
541            DiffMode::ImportGreaterUpdates | DiffMode::Linear => {
542                let changed = std::mem::take(&mut self.changed);
543                let mode = self.current_mode;
544                // Reset this field to avoid we use `has_all` to cache the diff calc and use it next round
545                // (In the next round we need to use the checkout mode)
546                self.current_mode = DiffMode::Checkout;
547                (InternalDiff::Map(MapDelta { updated: changed }), mode)
548            }
549        }
550    }
551}
552
553use rle::{HasLength as _, Sliceable};
554
555#[derive(Default)]
556pub(crate) struct ListDiffCalculator {
557    start_vv: VersionVector,
558    tracker: Box<RichtextTracker>,
559}
560
561impl ListDiffCalculator {
562    pub(crate) fn get_id_latest_pos(&self, id: ID) -> Option<crate::cursor::AbsolutePosition> {
563        self.tracker.get_target_id_latest_index_at_new_version(id)
564    }
565}
566
567impl MovableListDiffCalculator {
568    pub(crate) fn get_id_latest_pos(&self, id: ID) -> Option<crate::cursor::AbsolutePosition> {
569        self.list
570            .tracker
571            .get_target_id_latest_index_at_new_version(id)
572    }
573}
574
575impl std::fmt::Debug for ListDiffCalculator {
576    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
577        f.debug_struct("ListDiffCalculator")
578            // .field("tracker", &self.tracker)
579            .finish()
580    }
581}
582
583impl DiffCalculatorTrait for ListDiffCalculator {
584    fn start_tracking(&mut self, _oplog: &OpLog, vv: &crate::VersionVector, _mode: DiffMode) {
585        if !vv.includes_vv(&self.start_vv) || !self.tracker.all_vv().includes_vv(vv) {
586            self.tracker = Box::new(RichtextTracker::new_with_unknown());
587            self.start_vv = vv.clone();
588        }
589
590        self.tracker.checkout(vv);
591    }
592
593    fn apply_change(
594        &mut self,
595        _oplog: &OpLog,
596        op: crate::op::RichOp,
597        vv: Option<&crate::VersionVector>,
598    ) {
599        if let Some(vv) = vv {
600            self.tracker.checkout(vv);
601        }
602
603        match &op.op().content {
604            crate::op::InnerContent::List(l) => match l {
605                InnerListOp::Insert { slice, pos } => {
606                    self.tracker.insert(
607                        op.id_full(),
608                        *pos,
609                        RichtextChunk::new_text(slice.0.clone()),
610                    );
611                }
612                InnerListOp::Delete(del) => {
613                    self.tracker.delete(
614                        op.id_start(),
615                        del.id_start,
616                        del.start() as usize,
617                        del.atom_len(),
618                        del.is_reversed(),
619                    );
620                }
621                _ => unreachable!(),
622            },
623            _ => unreachable!(),
624        }
625    }
626
627    fn finish_this_round(&mut self) {}
628
629    fn calculate_diff(
630        &mut self,
631        idx: ContainerIdx,
632        oplog: &OpLog,
633        info: DiffCalcVersionInfo,
634        mut on_new_container: impl FnMut(&ContainerID),
635    ) -> (InternalDiff, DiffMode) {
636        let mut delta = Delta::new();
637        for item in self.tracker.diff(info.from_vv, info.to_vv) {
638            match item {
639                CrdtRopeDelta::Retain(len) => {
640                    delta = delta.retain(len);
641                }
642                CrdtRopeDelta::Insert {
643                    chunk: value,
644                    id,
645                    lamport,
646                } => match value.value() {
647                    RichtextChunkValue::Text(range) => {
648                        for i in range.clone() {
649                            let v = oplog.arena.get_value(i as usize);
650                            if let Some(LoroValue::Container(c)) = &v {
651                                on_new_container(c);
652                            }
653                        }
654                        delta = delta.insert(SliceWithId {
655                            values: Either::Left(SliceRange(range)),
656                            id: IdFull::new(id.peer, id.counter, lamport.unwrap()),
657                            elem_id: None,
658                        });
659                    }
660                    RichtextChunkValue::StyleAnchor { .. } => unreachable!(),
661                    RichtextChunkValue::Unknown(len) => {
662                        delta = handle_unknown(idx, id, oplog, len, &mut on_new_container, delta);
663                    }
664                    RichtextChunkValue::MoveAnchor => {
665                        delta = handle_unknown(idx, id, oplog, 1, &mut on_new_container, delta);
666                    }
667                },
668                CrdtRopeDelta::Delete(len) => {
669                    delta = delta.delete(len);
670                }
671            }
672        }
673
674        /// Handle span with unknown content when calculating diff
675        ///
676        /// We can lookup the content of the span by the id in the oplog
677        fn handle_unknown(
678            idx: ContainerIdx,
679            mut id: ID,
680            oplog: &OpLog,
681            len: u32,
682            on_new_container: &mut dyn FnMut(&ContainerID),
683            mut delta: Delta<SliceWithId>,
684        ) -> Delta<SliceWithId> {
685            // assert not unknown id
686            assert_ne!(id.peer, PeerID::MAX);
687            let mut acc_len = 0;
688            let end = id.counter + len as Counter;
689            let shallow_root = oplog.shallow_since_vv().get(&id.peer).copied().unwrap_or(0);
690            if id.counter < shallow_root {
691                // need to find the content between id.counter ~ target_end in gc state
692                let target_end = shallow_root.min(end);
693                delta = oplog.with_history_cache(|h| {
694                    let chunks =
695                        h.find_list_chunks_in(idx, IdSpan::new(id.peer, id.counter, target_end));
696                    for c in chunks {
697                        acc_len += c.length();
698                        match &c.values {
699                            Either::Left(_) => unreachable!(),
700                            Either::Right(r) => {
701                                if let LoroValue::Container(c) = r {
702                                    on_new_container(c)
703                                }
704                            }
705                        }
706                        delta = delta.insert(c);
707                    }
708
709                    delta
710                });
711                id.counter = shallow_root;
712            }
713
714            if id.counter < end {
715                for rich_op in oplog.iter_ops(IdSpan::new(id.peer, id.counter, end)) {
716                    acc_len += rich_op.content_len();
717                    let op = rich_op.op();
718                    let lamport = rich_op.lamport();
719
720                    if let InnerListOp::Insert { slice, pos: _ } = op.content.as_list().unwrap() {
721                        let range = slice.clone();
722                        for i in slice.0.clone() {
723                            let v = oplog.arena.get_value(i as usize);
724                            if let Some(LoroValue::Container(c)) = &v {
725                                (on_new_container)(c);
726                            }
727                        }
728
729                        delta = delta.insert(SliceWithId {
730                            values: Either::Left(range),
731                            id: IdFull::new(id.peer, op.counter, lamport),
732                            elem_id: None,
733                        });
734                    } else if let InnerListOp::Move { elem_id, .. } = op.content.as_list().unwrap()
735                    {
736                        delta = delta.insert(SliceWithId {
737                            // We do NOT need an actual value range,
738                            // movable list container will only use the id info
739                            values: Either::Right(LoroValue::Null),
740                            id: IdFull::new(id.peer, op.counter, lamport),
741                            elem_id: Some(elem_id.compact()),
742                        });
743                    }
744                }
745            }
746
747            debug_assert_eq!(acc_len, len as usize);
748            delta
749        }
750
751        (InternalDiff::ListRaw(delta), DiffMode::Checkout)
752    }
753}
754
755#[derive(Debug)]
756pub(crate) struct RichtextDiffCalculator {
757    mode: Box<RichtextCalcMode>,
758}
759
760#[derive(Debug)]
761enum RichtextCalcMode {
762    Crdt {
763        tracker: Box<RichtextTracker>,
764        /// (op, end_pos)
765        styles: Vec<(StyleOp, usize)>,
766        start_vv: VersionVector,
767    },
768    Linear {
769        diff: DeltaRope<RichtextStateChunk, ()>,
770        last_style_start: Option<(Arc<StyleOp>, u32)>,
771    },
772}
773
774impl RichtextDiffCalculator {
775    pub fn new() -> Self {
776        Self {
777            mode: Box::new(RichtextCalcMode::Crdt {
778                tracker: Box::new(RichtextTracker::new_with_unknown()),
779                styles: Vec::new(),
780                start_vv: VersionVector::new(),
781            }),
782        }
783    }
784
785    /// This should be called after calc_diff
786    ///
787    /// TODO: Refactor, this can be simplified
788    pub fn get_id_latest_pos(&self, id: ID) -> Option<AbsolutePosition> {
789        match &*self.mode {
790            RichtextCalcMode::Crdt { tracker, .. } => {
791                tracker.get_target_id_latest_index_at_new_version(id)
792            }
793            RichtextCalcMode::Linear { .. } => unreachable!(),
794        }
795    }
796}
797
798impl DiffCalculatorTrait for RichtextDiffCalculator {
799    fn start_tracking(
800        &mut self,
801        _oplog: &super::oplog::OpLog,
802        vv: &crate::VersionVector,
803        mode: DiffMode,
804    ) {
805        match mode {
806            DiffMode::Linear => {
807                self.mode = Box::new(RichtextCalcMode::Linear {
808                    diff: DeltaRope::new(),
809                    last_style_start: None,
810                });
811            }
812            _ => {
813                if !matches!(&*self.mode, RichtextCalcMode::Crdt { .. }) {
814                    unreachable!();
815                }
816            }
817        }
818
819        match &mut *self.mode {
820            RichtextCalcMode::Crdt {
821                tracker,
822                styles,
823                start_vv,
824            } => {
825                if !vv.includes_vv(start_vv) || !tracker.all_vv().includes_vv(vv) {
826                    *tracker = Box::new(RichtextTracker::new_with_unknown());
827                    styles.clear();
828                    *start_vv = vv.clone();
829                }
830
831                tracker.checkout(vv);
832            }
833            RichtextCalcMode::Linear { .. } => {}
834        }
835    }
836
837    fn apply_change(
838        &mut self,
839        oplog: &super::oplog::OpLog,
840        op: crate::op::RichOp,
841        vv: Option<&crate::VersionVector>,
842    ) {
843        match &mut *self.mode {
844            RichtextCalcMode::Linear {
845                diff,
846                last_style_start,
847            } => match &op.raw_op().content {
848                crate::op::InnerContent::List(l) => match l {
849                    InnerListOp::Insert { .. }
850                    | InnerListOp::Move { .. }
851                    | InnerListOp::Set { .. } => {
852                        unreachable!()
853                    }
854                    InnerListOp::InsertText {
855                        slice: _,
856                        unicode_start,
857                        unicode_len: len,
858                        pos,
859                    } => {
860                        let s = oplog.arena.slice_by_unicode(
861                            *unicode_start as usize..(*unicode_start + *len) as usize,
862                        );
863                        diff.insert_value(
864                            *pos as usize,
865                            RichtextStateChunk::new_text(s, op.id_full()),
866                            (),
867                        );
868                    }
869                    InnerListOp::Delete(del) => {
870                        diff.delete(del.start() as usize, del.atom_len());
871                    }
872                    InnerListOp::StyleStart {
873                        start,
874                        end,
875                        key,
876                        info,
877                        value,
878                    } => {
879                        debug_assert!(start < end, "start: {}, end: {}", start, end);
880                        let style_op = Arc::new(StyleOp {
881                            lamport: op.lamport(),
882                            peer: op.peer,
883                            cnt: op.id_start().counter,
884                            key: key.clone(),
885                            value: value.clone(),
886                            info: *info,
887                        });
888
889                        *last_style_start = Some((style_op.clone(), *end));
890                        diff.insert_value(
891                            *start as usize,
892                            RichtextStateChunk::new_style(style_op, AnchorType::Start),
893                            (),
894                        );
895                    }
896                    InnerListOp::StyleEnd => {
897                        let (style_op, pos) = match last_style_start.take() {
898                            Some((style_op, pos)) => (style_op, pos),
899                            None => {
900                                let Some(start_op) = oplog.get_op_that_includes(op.id().inc(-1))
901                                else {
902                                    panic!("Unhandled checkout case")
903                                };
904
905                                let InnerListOp::StyleStart {
906                                    key,
907                                    value,
908                                    info,
909                                    end,
910                                    ..
911                                } = start_op.content.as_list().unwrap()
912                                else {
913                                    unreachable!()
914                                };
915                                let style_op = Arc::new(StyleOp {
916                                    lamport: op.lamport() - 1,
917                                    peer: op.peer,
918                                    cnt: op.id_start().counter - 1,
919                                    key: key.clone(),
920                                    value: value.clone(),
921                                    info: *info,
922                                });
923
924                                (style_op, *end)
925                            }
926                        };
927                        assert_eq!(style_op.peer, op.peer);
928                        assert_eq!(style_op.cnt, op.id_start().counter - 1);
929                        diff.insert_value(
930                            pos as usize + 1,
931                            RichtextStateChunk::new_style(style_op, AnchorType::End),
932                            (),
933                        );
934                    }
935                },
936                _ => unreachable!(),
937            },
938            RichtextCalcMode::Crdt {
939                tracker,
940                styles,
941                start_vv: _,
942            } => {
943                if let Some(vv) = vv {
944                    tracker.checkout(vv);
945                }
946                match &op.raw_op().content {
947                    crate::op::InnerContent::List(l) => match l {
948                        InnerListOp::Insert { .. }
949                        | InnerListOp::Move { .. }
950                        | InnerListOp::Set { .. } => {
951                            unreachable!()
952                        }
953                        InnerListOp::InsertText {
954                            slice: _,
955                            unicode_start,
956                            unicode_len: len,
957                            pos,
958                        } => {
959                            tracker.insert(
960                                op.id_full(),
961                                *pos as usize,
962                                RichtextChunk::new_text(*unicode_start..*unicode_start + *len),
963                            );
964                        }
965                        InnerListOp::Delete(del) => {
966                            tracker.delete(
967                                op.id_start(),
968                                del.id_start,
969                                del.start() as usize,
970                                del.atom_len(),
971                                del.is_reversed(),
972                            );
973                        }
974                        InnerListOp::StyleStart {
975                            start,
976                            end,
977                            key,
978                            info,
979                            value,
980                        } => {
981                            debug_assert!(start < end, "start: {}, end: {}", start, end);
982                            let style_id = styles.len();
983                            styles.push((
984                                StyleOp {
985                                    lamport: op.lamport(),
986                                    peer: op.peer,
987                                    cnt: op.id_start().counter,
988                                    key: key.clone(),
989                                    value: value.clone(),
990                                    info: *info,
991                                },
992                                *end as usize,
993                            ));
994                            tracker.insert(
995                                op.id_full(),
996                                *start as usize,
997                                RichtextChunk::new_style_anchor(style_id as u32, AnchorType::Start),
998                            );
999                        }
1000                        InnerListOp::StyleEnd => {
1001                            let id = op.id();
1002                            if let Some(pos) = styles.iter().rev().position(|(op, _pos)| {
1003                                op.peer == id.peer && op.cnt == id.counter - 1
1004                            }) {
1005                                let style_id = styles.len() - pos - 1;
1006                                let (_start_op, end_pos) = &styles[style_id];
1007                                tracker.insert(
1008                                    op.id_full(),
1009                                    // need to shift 1 because we insert the start style anchor before this pos
1010                                    *end_pos + 1,
1011                                    RichtextChunk::new_style_anchor(
1012                                        style_id as u32,
1013                                        AnchorType::End,
1014                                    ),
1015                                );
1016                            } else {
1017                                let Some(start_op) = oplog.get_op_that_includes(op.id().inc(-1))
1018                                else {
1019                                    // Checkout on richtext that export at a gc version that split
1020                                    // start style op and end style op apart. Won't fix for now.
1021                                    // It's such a rare case...
1022                                    unimplemented!("Unhandled checkout case")
1023                                };
1024                                let InnerListOp::StyleStart {
1025                                    start: _,
1026                                    end,
1027                                    key,
1028                                    value,
1029                                    info,
1030                                } = start_op.content.as_list().unwrap()
1031                                else {
1032                                    unreachable!()
1033                                };
1034
1035                                styles.push((
1036                                    StyleOp {
1037                                        lamport: op.lamport() - 1,
1038                                        peer: id.peer,
1039                                        cnt: id.counter - 1,
1040                                        key: key.clone(),
1041                                        value: value.clone(),
1042                                        info: *info,
1043                                    },
1044                                    *end as usize,
1045                                ));
1046                                let style_id = styles.len() - 1;
1047                                tracker.insert(
1048                                    op.id_full(),
1049                                    // need to shift 1 because we insert the start style anchor before this pos
1050                                    *end as usize + 1,
1051                                    RichtextChunk::new_style_anchor(
1052                                        style_id as u32,
1053                                        AnchorType::End,
1054                                    ),
1055                                );
1056                            }
1057                        }
1058                    },
1059                    _ => unreachable!(),
1060                }
1061            }
1062        }
1063    }
1064
1065    fn calculate_diff(
1066        &mut self,
1067        idx: ContainerIdx,
1068        oplog: &OpLog,
1069        info: DiffCalcVersionInfo,
1070        _: impl FnMut(&ContainerID),
1071    ) -> (InternalDiff, DiffMode) {
1072        match &mut *self.mode {
1073            RichtextCalcMode::Linear { diff, .. } => (
1074                InternalDiff::RichtextRaw(std::mem::take(diff)),
1075                DiffMode::Linear,
1076            ),
1077            RichtextCalcMode::Crdt {
1078                tracker, styles, ..
1079            } => {
1080                let mut delta = DeltaRope::new();
1081                for item in tracker.diff(info.from_vv, info.to_vv) {
1082                    match item {
1083                        CrdtRopeDelta::Retain(len) => {
1084                            delta.push_retain(len, ());
1085                        }
1086                        CrdtRopeDelta::Insert {
1087                            chunk: value,
1088                            id,
1089                            lamport,
1090                        } => match value.value() {
1091                            RichtextChunkValue::Text(text) => {
1092                                delta.push_insert(
1093                                    RichtextStateChunk::Text(
1094                                        // PERF: can be speedup by acquiring lock on arena
1095                                        TextChunk::new(
1096                                            oplog.arena.slice_by_unicode(
1097                                                text.start as usize..text.end as usize,
1098                                            ),
1099                                            IdFull::new(id.peer, id.counter, lamport.unwrap()),
1100                                        ),
1101                                    ),
1102                                    (),
1103                                );
1104                            }
1105                            RichtextChunkValue::StyleAnchor { id, anchor_type } => {
1106                                delta.push_insert(
1107                                    RichtextStateChunk::Style {
1108                                        style: Arc::new(styles[id as usize].0.clone()),
1109                                        anchor_type,
1110                                    },
1111                                    (),
1112                                );
1113                            }
1114                            RichtextChunkValue::Unknown(len) => {
1115                                // assert not unknown id
1116                                assert_ne!(id.peer, PeerID::MAX);
1117                                let mut id = id;
1118                                let mut acc_len = 0;
1119                                let end = id.counter + len as Counter;
1120                                let shallow_root =
1121                                    oplog.shallow_since_vv().get(&id.peer).copied().unwrap_or(0);
1122                                if id.counter < shallow_root {
1123                                    // need to find the content between id.counter ~ target_end in gc state
1124                                    let target_end = shallow_root.min(end);
1125                                    oplog.with_history_cache(|h| {
1126                                        let chunks = h.find_text_chunks_in(
1127                                            idx,
1128                                            IdSpan::new(id.peer, id.counter, target_end),
1129                                        );
1130                                        for c in chunks {
1131                                            acc_len += c.rle_len();
1132                                            delta.push_insert(c, ());
1133                                        }
1134                                    });
1135                                    id.counter = shallow_root;
1136                                }
1137
1138                                if id.counter < end {
1139                                    for rich_op in
1140                                        oplog.iter_ops(IdSpan::new(id.peer, id.counter, end))
1141                                    {
1142                                        acc_len += rich_op.content_len();
1143                                        let op = rich_op.op();
1144                                        let lamport = rich_op.lamport();
1145                                        let content = op.content.as_list().unwrap();
1146                                        match content {
1147                                            InnerListOp::InsertText { slice, .. } => {
1148                                                delta.push_insert(
1149                                                    RichtextStateChunk::Text(TextChunk::new(
1150                                                        slice.clone(),
1151                                                        IdFull::new(id.peer, op.counter, lamport),
1152                                                    )),
1153                                                    (),
1154                                                );
1155                                            }
1156                                            _ => unreachable!("{:?}", content),
1157                                        }
1158                                    }
1159                                }
1160
1161                                debug_assert_eq!(acc_len, len as usize);
1162                            }
1163                            RichtextChunkValue::MoveAnchor => unreachable!(),
1164                        },
1165                        CrdtRopeDelta::Delete(len) => {
1166                            delta.push_delete(len);
1167                        }
1168                    }
1169                }
1170
1171                (InternalDiff::RichtextRaw(delta), DiffMode::Checkout)
1172            }
1173        }
1174    }
1175
1176    fn finish_this_round(&mut self) {
1177        match &mut *self.mode {
1178            RichtextCalcMode::Crdt { .. } => {}
1179            RichtextCalcMode::Linear {
1180                diff,
1181                last_style_start,
1182            } => {
1183                *diff = DeltaRope::new();
1184                last_style_start.take();
1185            }
1186        }
1187    }
1188}
1189
1190#[derive(Debug)]
1191pub(crate) struct MovableListDiffCalculator {
1192    list: Box<ListDiffCalculator>,
1193    inner: Box<MovableListInner>,
1194}
1195
1196#[derive(Debug)]
1197struct MovableListInner {
1198    changed_elements: FxHashMap<CompactIdLp, ElementDelta>,
1199    move_id_to_elem_id: FxHashMap<ID, IdLp>,
1200    current_mode: DiffMode,
1201}
1202
1203impl DiffCalculatorTrait for MovableListDiffCalculator {
1204    fn start_tracking(&mut self, _oplog: &OpLog, vv: &crate::VersionVector, mode: DiffMode) {
1205        if !vv.includes_vv(&self.list.start_vv) || !self.list.tracker.all_vv().includes_vv(vv) {
1206            self.list.tracker = Box::new(RichtextTracker::new_with_unknown());
1207            self.list.start_vv = vv.clone();
1208        }
1209
1210        self.list.tracker.checkout(vv);
1211        self.inner.current_mode = mode;
1212    }
1213
1214    fn apply_change(
1215        &mut self,
1216        oplog: &OpLog,
1217        op: crate::op::RichOp,
1218        vv: Option<&crate::VersionVector>,
1219    ) {
1220        let InnerContent::List(l) = &op.raw_op().content else {
1221            unreachable!()
1222        };
1223
1224        // collect the elements that are moved, updated, or inserted
1225
1226        // If it's checkout mode, we don't need to track the changes
1227        // we only need the element ids
1228        match l {
1229            InnerListOp::Insert { slice, pos: _ } => {
1230                let op_id = op.id_full().idlp();
1231                for i in 0..slice.atom_len() {
1232                    let id = op_id.inc(i as Counter);
1233                    let value = oplog.arena.get_value(slice.0.start as usize + i).unwrap();
1234
1235                    self.inner.changed_elements.insert(
1236                        id.compact(),
1237                        ElementDelta {
1238                            pos: Some(id),
1239                            value: value.clone(),
1240                            value_updated: true,
1241                            value_id: Some(id),
1242                        },
1243                    );
1244                }
1245            }
1246            InnerListOp::Delete(_) => {}
1247            InnerListOp::Move { elem_id, .. } => {
1248                let idlp = IdLp::new(op.peer, op.lamport());
1249                match self.inner.changed_elements.get_mut(&elem_id.compact()) {
1250                    Some(change) => {
1251                        if change.pos.is_some() && change.pos.as_ref().unwrap() > &idlp {
1252                        } else {
1253                            change.pos = Some(idlp);
1254                        }
1255                    }
1256                    None => {
1257                        self.inner.changed_elements.insert(
1258                            elem_id.compact(),
1259                            ElementDelta {
1260                                pos: Some(idlp),
1261                                value: LoroValue::Null,
1262                                value_updated: false,
1263                                value_id: None,
1264                            },
1265                        );
1266                    }
1267                }
1268            }
1269            InnerListOp::Set { elem_id, value } => {
1270                let idlp = IdLp::new(op.peer, op.lamport());
1271                match self.inner.changed_elements.get_mut(&elem_id.compact()) {
1272                    Some(change) => {
1273                        if change.value_id.is_some() && change.value_id.as_ref().unwrap() > &idlp {
1274                        } else {
1275                            change.value_id = Some(idlp);
1276                            change.value = value.clone();
1277                        }
1278                    }
1279                    None => {
1280                        self.inner.changed_elements.insert(
1281                            elem_id.compact(),
1282                            ElementDelta {
1283                                pos: None,
1284                                value: value.clone(),
1285                                value_updated: true,
1286                                value_id: Some(idlp),
1287                            },
1288                        );
1289                    }
1290                }
1291            }
1292
1293            InnerListOp::StyleStart { .. } => unreachable!(),
1294            InnerListOp::StyleEnd => unreachable!(),
1295            InnerListOp::InsertText { .. } => unreachable!(),
1296        }
1297
1298        let is_checkout = matches!(self.inner.current_mode, DiffMode::Checkout);
1299
1300        {
1301            // Apply change on the list items
1302            let this = &mut self.list;
1303            if let Some(vv) = vv {
1304                this.tracker.checkout(vv);
1305            }
1306
1307            let real_op = op.op();
1308            match &real_op.content {
1309                crate::op::InnerContent::List(l) => match l {
1310                    InnerListOp::Insert { slice, pos } => {
1311                        this.tracker.insert(
1312                            op.id_full(),
1313                            *pos,
1314                            RichtextChunk::new_text(slice.0.clone()),
1315                        );
1316                    }
1317                    InnerListOp::Delete(del) => {
1318                        this.tracker.delete(
1319                            op.id_start(),
1320                            del.id_start,
1321                            del.start() as usize,
1322                            del.atom_len(),
1323                            del.is_reversed(),
1324                        );
1325                    }
1326                    InnerListOp::Move { from, elem_id, to } => {
1327                        self.inner.move_id_to_elem_id.insert(op.id(), *elem_id);
1328                        if !this.tracker.current_vv().includes_id(op.id()) {
1329                            let last_pos = if is_checkout {
1330                                // TODO: PERF: this lookup can be optimized
1331                                oplog.with_history_cache(|h| {
1332                                    let list = &h.get_checkout_index().movable_list;
1333                                    list.last_pos(
1334                                        *elem_id,
1335                                        this.tracker.current_vv(),
1336                                        // TODO: PERF: Provide the lamport of to version
1337                                        Lamport::MAX,
1338                                        oplog,
1339                                    )
1340                                    .unwrap()
1341                                    .id()
1342                                })
1343                            } else {
1344                                // When it's import or linear mode, we need to use a fake id
1345                                // because we want to avoid using the history cache
1346                                //
1347                                // This ID will not be used. Because it will only be used when
1348                                // we switch to an older version. And we know it's for importing and
1349                                // to version is always after from version (!is_checkout), so that
1350                                // we don't need to checkout to the version before from.
1351                                const FAKE_ID: ID = ID {
1352                                    peer: PeerID::MAX - 2,
1353                                    counter: 0,
1354                                };
1355                                FAKE_ID
1356                            };
1357                            this.tracker.move_item(
1358                                op.id_full(),
1359                                last_pos,
1360                                *from as usize,
1361                                *to as usize,
1362                            );
1363                        }
1364                    }
1365                    InnerListOp::Set { .. } => {
1366                        // don't need to update tracker here
1367                    }
1368                    InnerListOp::InsertText { .. }
1369                    | InnerListOp::StyleStart { .. }
1370                    | InnerListOp::StyleEnd => unreachable!(),
1371                },
1372                _ => unreachable!(),
1373            }
1374        };
1375    }
1376
1377    fn finish_this_round(&mut self) {
1378        self.list.finish_this_round();
1379    }
1380
1381    #[instrument(skip(self, oplog, on_new_container))]
1382    fn calculate_diff(
1383        &mut self,
1384        idx: ContainerIdx,
1385        oplog: &OpLog,
1386        info: DiffCalcVersionInfo,
1387        mut on_new_container: impl FnMut(&ContainerID),
1388    ) -> (InternalDiff, DiffMode) {
1389        let (InternalDiff::ListRaw(list_diff), diff_mode) =
1390            self.list.calculate_diff(idx, oplog, info, |_| {})
1391        else {
1392            unreachable!()
1393        };
1394
1395        assert_eq!(diff_mode, DiffMode::Checkout);
1396        let is_checkout = matches!(
1397            self.inner.current_mode,
1398            DiffMode::Checkout | DiffMode::Import
1399        );
1400        let mut element_changes: FxHashMap<CompactIdLp, ElementDelta> = if is_checkout {
1401            FxHashMap::default()
1402        } else {
1403            std::mem::take(&mut self.inner.changed_elements)
1404        };
1405
1406        if is_checkout {
1407            for id in self.inner.changed_elements.keys() {
1408                element_changes.insert(*id, ElementDelta::placeholder());
1409            }
1410        }
1411
1412        let list_diff: Delta<SmallVec<[IdFull; 1]>, ()> = Delta {
1413            vec: list_diff
1414                .iter()
1415                .map(|x| match x {
1416                    &DeltaItem::Retain { retain, .. } => DeltaItem::Retain {
1417                        retain,
1418                        attributes: (),
1419                    },
1420                    DeltaItem::Insert { insert, .. } => {
1421                        let len = insert.length();
1422                        let id = insert.id;
1423                        let mut new_insert = SmallVec::with_capacity(len);
1424                        for i in 0..len {
1425                            let id = id.inc(i as i32);
1426                            let elem_id =
1427                                if let Some(e) = self.inner.move_id_to_elem_id.get(&id.id()) {
1428                                    e.compact()
1429                                } else {
1430                                    insert.elem_id.unwrap_or_else(|| id.idlp().compact())
1431                                };
1432                            if is_checkout {
1433                                // add the related element id
1434                                element_changes.insert(elem_id, ElementDelta::placeholder());
1435                            }
1436                            new_insert.push(id);
1437                        }
1438
1439                        DeltaItem::Insert {
1440                            insert: new_insert,
1441                            attributes: (),
1442                        }
1443                    }
1444                    &DeltaItem::Delete { delete, .. } => DeltaItem::Delete {
1445                        delete,
1446                        attributes: (),
1447                    },
1448                })
1449                .collect(),
1450        };
1451
1452        if is_checkout {
1453            oplog.with_history_cache(|history_cache| {
1454                let checkout_index = &history_cache.get_checkout_index().movable_list;
1455                element_changes.retain(|id, change| {
1456                    let id = id.to_id();
1457                    // It can be None if the target does not exist before the `to` version
1458                    // But we don't need to calc from, because the deletion is handled by the diff from list items
1459
1460                    // TODO: PERF: Provide the lamport of to version
1461                    let Some(pos) = checkout_index.last_pos(id, info.to_vv, Lamport::MAX, oplog)
1462                    else {
1463                        return false;
1464                    };
1465                    // TODO: PERF: Provide the lamport of to version
1466                    let value = checkout_index
1467                        .last_value(id, info.to_vv, Lamport::MAX, oplog)
1468                        .unwrap();
1469                    // TODO: PERF: Provide the lamport of to version
1470                    let old_pos = checkout_index.last_pos(id, info.from_vv, Lamport::MAX, oplog);
1471                    // TODO: PERF: Provide the lamport of to version
1472                    let old_value =
1473                        checkout_index.last_value(id, info.from_vv, Lamport::MAX, oplog);
1474                    if old_pos.is_none() && old_value.is_none() {
1475                        if let LoroValue::Container(c) = &value.value {
1476                            on_new_container(c);
1477                        }
1478                        *change = ElementDelta {
1479                            pos: Some(pos.idlp()),
1480                            value: value.value.clone(),
1481                            value_id: Some(IdLp::new(value.peer, value.lamport)),
1482                            value_updated: true,
1483                        };
1484                    } else {
1485                        // TODO: PERF: can be filtered based on the list_diff and whether the pos/value are updated
1486                        *change = ElementDelta {
1487                            pos: Some(pos.idlp()),
1488                            value: value.value.clone(),
1489                            value_updated: old_value.unwrap().value != value.value,
1490                            value_id: Some(IdLp::new(value.peer, value.lamport)),
1491                        };
1492                    }
1493
1494                    true
1495                });
1496            });
1497        }
1498
1499        let diff = MovableListInnerDelta {
1500            list: list_diff,
1501            elements: element_changes,
1502        };
1503
1504        (InternalDiff::MovableList(diff), self.inner.current_mode)
1505    }
1506}
1507
1508impl MovableListDiffCalculator {
1509    fn new(_container: ContainerIdx) -> MovableListDiffCalculator {
1510        MovableListDiffCalculator {
1511            list: Default::default(),
1512            inner: Box::new(MovableListInner {
1513                changed_elements: Default::default(),
1514                current_mode: DiffMode::Checkout,
1515                move_id_to_elem_id: Default::default(),
1516            }),
1517        }
1518    }
1519}
1520
1521#[test]
1522fn test_size() {
1523    let text = RichtextDiffCalculator::new();
1524    let size = std::mem::size_of_val(&text);
1525    assert!(size < 50, "RichtextDiffCalculator size: {}", size);
1526    let list = MovableListDiffCalculator::new(ContainerIdx::from_index_and_type(
1527        0,
1528        loro_common::ContainerType::MovableList,
1529    ));
1530    let size = std::mem::size_of_val(&list);
1531    assert!(size < 50, "MovableListDiffCalculator size: {}", size);
1532    let calc = ContainerDiffCalculator::Richtext(text);
1533    let size = std::mem::size_of_val(&calc);
1534    assert!(size < 50, "ContainerDiffCalculator size: {}", size);
1535}