loro_internal/
state.rs

1use crate::sync::{AtomicU64, Mutex};
2use std::sync::RwLock;
3use std::sync::{Arc, Weak};
4use std::{borrow::Cow, io::Write, sync::atomic::Ordering};
5
6use container_store::ContainerStore;
7use dead_containers_cache::DeadContainersCache;
8use enum_as_inner::EnumAsInner;
9use enum_dispatch::enum_dispatch;
10use fxhash::{FxHashMap, FxHashSet};
11use itertools::Itertools;
12use loro_common::{ContainerID, LoroError, LoroResult, TreeID};
13use loro_delta::DeltaItem;
14use tracing::{info_span, instrument, warn};
15
16use crate::{
17    configure::{Configure, DefaultRandom, SecureRandomGenerator},
18    container::{idx::ContainerIdx, richtext::config::StyleConfigMap, ContainerIdRaw},
19    cursor::Cursor,
20    delta::TreeExternalDiff,
21    diff_calc::{DiffCalculator, DiffMode},
22    encoding::{StateSnapshotDecodeContext, StateSnapshotEncoder},
23    event::{Diff, EventTriggerKind, Index, InternalContainerDiff, InternalDiff},
24    fx_map,
25    handler::ValueOrHandler,
26    id::PeerID,
27    lock::{LoroLockGroup, LoroMutex},
28    op::{Op, RawOp},
29    version::Frontiers,
30    ContainerDiff, ContainerType, DocDiff, InternalString, LoroDocInner, LoroValue, OpLog,
31};
32
33pub(crate) mod analyzer;
34pub(crate) mod container_store;
35#[cfg(feature = "counter")]
36mod counter_state;
37mod dead_containers_cache;
38mod list_state;
39mod map_state;
40mod movable_list_state;
41mod richtext_state;
42mod tree_state;
43mod unknown_state;
44
45pub(crate) use self::movable_list_state::{IndexType, MovableListState};
46pub(crate) use container_store::GcStore;
47pub(crate) use list_state::ListState;
48pub(crate) use map_state::MapState;
49pub(crate) use richtext_state::RichtextState;
50pub(crate) use tree_state::FiIfNotConfigured;
51pub(crate) use tree_state::{get_meta_value, FractionalIndexGenResult, NodePosition, TreeState};
52pub use tree_state::{TreeNode, TreeNodeWithChildren, TreeParentId};
53
54use self::{container_store::ContainerWrapper, unknown_state::UnknownState};
55
56#[cfg(feature = "counter")]
57use self::counter_state::CounterState;
58
59use super::{arena::SharedArena, event::InternalDocDiff};
60
61pub struct DocState {
62    pub(super) peer: Arc<AtomicU64>,
63
64    pub(super) frontiers: Frontiers,
65    // pub(super) states: FxHashMap<ContainerIdx, State>,
66    pub(super) store: ContainerStore,
67    pub(super) arena: SharedArena,
68    pub(crate) config: Configure,
69    // resolve event stuff
70    doc: Weak<LoroDocInner>,
71    // txn related stuff
72    in_txn: bool,
73    changed_idx_in_txn: FxHashSet<ContainerIdx>,
74
75    // diff related stuff
76    event_recorder: EventRecorder,
77
78    dead_containers_cache: DeadContainersCache,
79}
80
81impl std::fmt::Debug for DocState {
82    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
83        f.debug_struct("DocState")
84            .field("peer", &self.peer)
85            .finish()
86    }
87}
88
89#[derive(Clone, Copy)]
90pub(crate) struct ContainerCreationContext<'a> {
91    pub configure: &'a Configure,
92    pub peer: PeerID,
93}
94
95pub(crate) struct DiffApplyContext<'a> {
96    pub mode: DiffMode,
97    pub doc: &'a Weak<LoroDocInner>,
98}
99
100pub(crate) trait FastStateSnapshot {
101    fn encode_snapshot_fast<W: Write>(&mut self, w: W);
102    fn decode_value(bytes: &[u8]) -> LoroResult<(LoroValue, &[u8])>;
103    fn decode_snapshot_fast(
104        idx: ContainerIdx,
105        v: (LoroValue, &[u8]),
106        ctx: ContainerCreationContext,
107    ) -> LoroResult<Self>
108    where
109        Self: Sized;
110}
111
112#[derive(Debug, Clone, Default)]
113pub(crate) struct ApplyLocalOpReturn {
114    pub deleted_containers: Vec<ContainerID>,
115}
116
117#[enum_dispatch]
118pub(crate) trait ContainerState {
119    fn container_idx(&self) -> ContainerIdx;
120    fn estimate_size(&self) -> usize;
121
122    fn is_state_empty(&self) -> bool;
123
124    #[must_use]
125    fn apply_diff_and_convert(&mut self, diff: InternalDiff, ctx: DiffApplyContext) -> Diff;
126
127    fn apply_diff(&mut self, diff: InternalDiff, ctx: DiffApplyContext);
128
129    fn apply_local_op(&mut self, raw_op: &RawOp, op: &Op) -> LoroResult<ApplyLocalOpReturn>;
130    /// Convert a state to a diff, such that an empty state will be transformed into the same as this state when it's applied.
131    fn to_diff(&mut self, doc: &Weak<LoroDocInner>) -> Diff;
132
133    fn get_value(&mut self) -> LoroValue;
134
135    /// Get the index of the child container
136    #[allow(unused)]
137    fn get_child_index(&self, id: &ContainerID) -> Option<Index>;
138
139    #[allow(unused)]
140    fn contains_child(&self, id: &ContainerID) -> bool;
141
142    #[allow(unused)]
143    fn get_child_containers(&self) -> Vec<ContainerID>;
144
145    /// Encode the ops and the blob that can be used to restore the state to the current state.
146    ///
147    /// State will use the provided encoder to encode the ops and export a blob.
148    /// The ops should be encoded into the snapshot as well as the blob.
149    /// The users then can use the ops and the blob to restore the state to the current state.
150    fn encode_snapshot(&self, encoder: StateSnapshotEncoder) -> Vec<u8>;
151
152    /// Restore the state to the state represented by the ops and the blob that exported by `get_snapshot_ops`
153    fn import_from_snapshot_ops(&mut self, ctx: StateSnapshotDecodeContext) -> LoroResult<()>;
154    fn fork(&self, config: &Configure) -> Self;
155}
156
157impl<T: FastStateSnapshot> FastStateSnapshot for Box<T> {
158    fn encode_snapshot_fast<W: Write>(&mut self, w: W) {
159        self.as_mut().encode_snapshot_fast(w)
160    }
161
162    fn decode_value(bytes: &[u8]) -> LoroResult<(LoroValue, &[u8])> {
163        T::decode_value(bytes)
164    }
165
166    fn decode_snapshot_fast(
167        idx: ContainerIdx,
168        v: (LoroValue, &[u8]),
169        ctx: ContainerCreationContext,
170    ) -> LoroResult<Self>
171    where
172        Self: Sized,
173    {
174        T::decode_snapshot_fast(idx, v, ctx).map(|x| Box::new(x))
175    }
176}
177
178impl<T: ContainerState> ContainerState for Box<T> {
179    fn container_idx(&self) -> ContainerIdx {
180        self.as_ref().container_idx()
181    }
182
183    fn estimate_size(&self) -> usize {
184        self.as_ref().estimate_size()
185    }
186
187    fn is_state_empty(&self) -> bool {
188        self.as_ref().is_state_empty()
189    }
190
191    fn apply_diff_and_convert(&mut self, diff: InternalDiff, ctx: DiffApplyContext) -> Diff {
192        self.as_mut().apply_diff_and_convert(diff, ctx)
193    }
194
195    fn apply_diff(&mut self, diff: InternalDiff, ctx: DiffApplyContext) {
196        self.as_mut().apply_diff(diff, ctx)
197    }
198
199    fn apply_local_op(&mut self, raw_op: &RawOp, op: &Op) -> LoroResult<ApplyLocalOpReturn> {
200        self.as_mut().apply_local_op(raw_op, op)
201    }
202
203    #[doc = r" Convert a state to a diff, such that an empty state will be transformed into the same as this state when it's applied."]
204    fn to_diff(&mut self, doc: &Weak<LoroDocInner>) -> Diff {
205        self.as_mut().to_diff(doc)
206    }
207
208    fn get_value(&mut self) -> LoroValue {
209        self.as_mut().get_value()
210    }
211
212    #[doc = r" Get the index of the child container"]
213    #[allow(unused)]
214    fn get_child_index(&self, id: &ContainerID) -> Option<Index> {
215        self.as_ref().get_child_index(id)
216    }
217
218    fn contains_child(&self, id: &ContainerID) -> bool {
219        self.as_ref().contains_child(id)
220    }
221
222    #[allow(unused)]
223    fn get_child_containers(&self) -> Vec<ContainerID> {
224        self.as_ref().get_child_containers()
225    }
226
227    #[doc = r" Encode the ops and the blob that can be used to restore the state to the current state."]
228    #[doc = r""]
229    #[doc = r" State will use the provided encoder to encode the ops and export a blob."]
230    #[doc = r" The ops should be encoded into the snapshot as well as the blob."]
231    #[doc = r" The users then can use the ops and the blob to restore the state to the current state."]
232    fn encode_snapshot(&self, encoder: StateSnapshotEncoder) -> Vec<u8> {
233        self.as_ref().encode_snapshot(encoder)
234    }
235
236    #[doc = r" Restore the state to the state represented by the ops and the blob that exported by `get_snapshot_ops`"]
237    fn import_from_snapshot_ops(&mut self, ctx: StateSnapshotDecodeContext) -> LoroResult<()> {
238        self.as_mut().import_from_snapshot_ops(ctx)
239    }
240
241    fn fork(&self, config: &Configure) -> Self {
242        Box::new(self.as_ref().fork(config))
243    }
244}
245
246#[allow(clippy::enum_variant_names)]
247#[enum_dispatch(ContainerState)]
248#[derive(EnumAsInner, Debug)]
249pub enum State {
250    ListState(Box<ListState>),
251    MovableListState(Box<MovableListState>),
252    MapState(Box<MapState>),
253    RichtextState(Box<RichtextState>),
254    TreeState(Box<TreeState>),
255    #[cfg(feature = "counter")]
256    CounterState(Box<counter_state::CounterState>),
257    UnknownState(UnknownState),
258}
259
260impl From<ListState> for State {
261    fn from(s: ListState) -> Self {
262        Self::ListState(Box::new(s))
263    }
264}
265
266impl From<RichtextState> for State {
267    fn from(s: RichtextState) -> Self {
268        Self::RichtextState(Box::new(s))
269    }
270}
271
272impl From<MovableListState> for State {
273    fn from(s: MovableListState) -> Self {
274        Self::MovableListState(Box::new(s))
275    }
276}
277
278impl From<MapState> for State {
279    fn from(s: MapState) -> Self {
280        Self::MapState(Box::new(s))
281    }
282}
283
284impl From<TreeState> for State {
285    fn from(s: TreeState) -> Self {
286        Self::TreeState(Box::new(s))
287    }
288}
289
290#[cfg(feature = "counter")]
291impl From<CounterState> for State {
292    fn from(s: CounterState) -> Self {
293        Self::CounterState(Box::new(s))
294    }
295}
296
297impl State {
298    pub fn new_list(idx: ContainerIdx) -> Self {
299        Self::ListState(Box::new(ListState::new(idx)))
300    }
301
302    pub fn new_map(idx: ContainerIdx) -> Self {
303        Self::MapState(Box::new(MapState::new(idx)))
304    }
305
306    pub fn new_richtext(idx: ContainerIdx, config: Arc<RwLock<StyleConfigMap>>) -> Self {
307        Self::RichtextState(Box::new(RichtextState::new(idx, config)))
308    }
309
310    pub fn new_tree(idx: ContainerIdx, peer: PeerID) -> Self {
311        Self::TreeState(Box::new(TreeState::new(idx, peer)))
312    }
313
314    pub fn new_unknown(idx: ContainerIdx) -> Self {
315        Self::UnknownState(UnknownState::new(idx))
316    }
317
318    pub fn encode_snapshot_fast<W: Write>(&mut self, mut w: W) {
319        match self {
320            State::ListState(s) => s.encode_snapshot_fast(&mut w),
321            State::MovableListState(s) => s.encode_snapshot_fast(&mut w),
322            State::MapState(s) => s.encode_snapshot_fast(&mut w),
323            State::RichtextState(s) => s.encode_snapshot_fast(&mut w),
324            State::TreeState(s) => s.encode_snapshot_fast(&mut w),
325            #[cfg(feature = "counter")]
326            State::CounterState(s) => s.encode_snapshot_fast(&mut w),
327            State::UnknownState(s) => s.encode_snapshot_fast(&mut w),
328        }
329    }
330
331    pub fn fork(&self, config: &Configure) -> Self {
332        match self {
333            State::ListState(list_state) => State::ListState(list_state.fork(config)),
334            State::MovableListState(movable_list_state) => {
335                State::MovableListState(movable_list_state.fork(config))
336            }
337            State::MapState(map_state) => State::MapState(map_state.fork(config)),
338            State::RichtextState(richtext_state) => {
339                State::RichtextState(richtext_state.fork(config))
340            }
341            State::TreeState(tree_state) => State::TreeState(tree_state.fork(config)),
342            #[cfg(feature = "counter")]
343            State::CounterState(counter_state) => State::CounterState(counter_state.fork(config)),
344            State::UnknownState(unknown_state) => State::UnknownState(unknown_state.fork(config)),
345        }
346    }
347}
348
349impl DocState {
350    #[inline]
351    pub fn new_arc(
352        doc: Weak<LoroDocInner>,
353        arena: SharedArena,
354        config: Configure,
355        lock_group: &LoroLockGroup,
356    ) -> Arc<LoroMutex<Self>> {
357        let peer = DefaultRandom.next_u64();
358        // TODO: maybe we should switch to certain version in oplog?
359
360        let peer = Arc::new(AtomicU64::new(peer));
361        Arc::new(lock_group.new_lock(
362            Self {
363                store: ContainerStore::new(arena.clone(), config.clone(), peer.clone()),
364                peer,
365                arena,
366                frontiers: Frontiers::default(),
367                doc,
368                config,
369                in_txn: false,
370                changed_idx_in_txn: FxHashSet::default(),
371                event_recorder: Default::default(),
372                dead_containers_cache: Default::default(),
373            },
374            crate::lock::LockKind::DocState,
375        ))
376    }
377
378    pub fn fork_with_new_peer_id(
379        &mut self,
380        doc: Weak<LoroDocInner>,
381        arena: SharedArena,
382        config: Configure,
383    ) -> Arc<Mutex<Self>> {
384        let peer = Arc::new(AtomicU64::new(DefaultRandom.next_u64()));
385        let store = self.store.fork(arena.clone(), peer.clone(), config.clone());
386        Arc::new(Mutex::new(Self {
387            peer,
388            frontiers: self.frontiers.clone(),
389            store,
390            arena,
391            config,
392            doc,
393            in_txn: false,
394            changed_idx_in_txn: FxHashSet::default(),
395            event_recorder: Default::default(),
396            dead_containers_cache: Default::default(),
397        }))
398    }
399
400    pub fn start_recording(&mut self) {
401        if self.is_recording() {
402            return;
403        }
404
405        self.event_recorder.recording_diff = true;
406        self.event_recorder.diff_start_version = Some(self.frontiers.clone());
407    }
408
409    #[inline(always)]
410    pub fn stop_and_clear_recording(&mut self) {
411        self.event_recorder = Default::default();
412    }
413
414    #[inline(always)]
415    pub fn is_recording(&self) -> bool {
416        self.event_recorder.recording_diff
417    }
418
419    pub fn refresh_peer_id(&mut self) {
420        self.peer.store(
421            DefaultRandom.next_u64(),
422            std::sync::atomic::Ordering::Relaxed,
423        );
424    }
425
426    /// Take all the diffs that are recorded and convert them to events.
427    pub fn take_events(&mut self) -> Vec<DocDiff> {
428        if !self.is_recording() {
429            return vec![];
430        }
431
432        self.convert_current_batch_diff_into_event();
433        std::mem::take(&mut self.event_recorder.events)
434    }
435
436    /// Record the next diff.
437    /// Caller should call [pre_txn] before calling this.
438    ///
439    /// # Panic
440    ///
441    /// Panic when the diff cannot be merged with the previous diff.
442    /// Caller should call [pre_txn] before calling this to avoid panic.
443    fn record_diff(&mut self, diff: InternalDocDiff) {
444        if !self.event_recorder.recording_diff || diff.diff.is_empty() {
445            return;
446        }
447
448        let Some(last_diff) = self.event_recorder.diffs.last_mut() else {
449            self.event_recorder.diffs.push(diff.into_owned());
450            return;
451        };
452
453        if last_diff.can_merge(&diff) {
454            self.event_recorder.diffs.push(diff.into_owned());
455            return;
456        }
457
458        panic!("should call pre_txn before record_diff")
459    }
460
461    /// This should be called when DocState is going to apply a transaction / a diff.
462    fn pre_txn(&mut self, next_origin: InternalString, next_trigger: EventTriggerKind) {
463        if !self.is_recording() {
464            return;
465        }
466
467        let Some(last_diff) = self.event_recorder.diffs.last() else {
468            return;
469        };
470
471        if last_diff.origin == next_origin && last_diff.by == next_trigger {
472            return;
473        }
474
475        // current diff batch cannot merge with the incoming diff,
476        // need to convert all the current diffs into event
477        self.convert_current_batch_diff_into_event()
478    }
479
480    fn convert_current_batch_diff_into_event(&mut self) {
481        let recorder = &mut self.event_recorder;
482        if recorder.diffs.is_empty() {
483            return;
484        }
485
486        let diffs = std::mem::take(&mut recorder.diffs);
487        let start = recorder.diff_start_version.take().unwrap();
488        recorder.diff_start_version = Some((*diffs.last().unwrap().new_version).to_owned());
489        let event = self.diffs_to_event(diffs, start);
490        self.event_recorder.events.push(event);
491    }
492
493    /// Change the peer id of this doc state.
494    /// It changes the peer id for the future txn on this AppState
495    #[inline]
496    pub fn set_peer_id(&mut self, peer: PeerID) {
497        self.peer.store(peer, std::sync::atomic::Ordering::Relaxed);
498    }
499
500    pub fn peer_id(&self) -> PeerID {
501        self.peer.load(std::sync::atomic::Ordering::Relaxed)
502    }
503
504    /// It's expected that diff only contains [`InternalDiff`]
505    ///
506    #[instrument(skip_all)]
507    pub(crate) fn apply_diff(&mut self, mut diff: InternalDocDiff<'static>, diff_mode: DiffMode) {
508        if self.in_txn {
509            panic!("apply_diff should not be called in a transaction");
510        }
511
512        match diff_mode {
513            DiffMode::Checkout => {
514                self.dead_containers_cache.clear();
515            }
516            _ => {
517                self.dead_containers_cache.clear_alive();
518            }
519        }
520
521        let is_recording = self.is_recording();
522        self.pre_txn(diff.origin.clone(), diff.by);
523        let Cow::Owned(mut diffs) = std::mem::take(&mut diff.diff) else {
524            unreachable!()
525        };
526
527        // # Revival
528        //
529        // A Container, if it is deleted from its parent Container, will still exist
530        // in the internal state of Loro;  whereas on the user side, a tree structure
531        // is maintained following Events, and at this point, the corresponding state
532        // is considered deleted.
533        //
534        // Sometimes, this "pseudo-dead" Container may be revived (for example, through
535        // backtracking or parallel editing),  and the user side should receive an Event
536        // that restores the consistency between the revived Container and the  internal
537        // state of Loro. This Event is required to restore the pseudo-dead  Container
538        // State to its current state on Loro, and we refer to this process as "revival".
539        //
540        // Revival occurs during the application of the internal diff, and this operation
541        // is necessary when it needs to be converted into an external Event.
542        //
543        // We can utilize the output of the Diff to determine which child nodes should be revived.
544        //
545        // For nodes that are to be revived, we can disregard the Events output by their
546        // round of apply_diff_and_convert,  and instead, directly convert their state into
547        // an Event once their application is complete.
548        //
549        // Suppose A is revived and B is A's child, and B also needs to be revived; therefore,
550        // we should process each level alternately.
551
552        // We need to ensure diff is processed in order
553        diffs.sort_by_cached_key(|diff| self.arena.get_depth(diff.idx));
554        let mut to_revive_in_next_layer: FxHashSet<ContainerIdx> = FxHashSet::default();
555        let mut to_revive_in_this_layer: FxHashSet<ContainerIdx> = FxHashSet::default();
556        let mut last_depth = 0;
557        let len = diffs.len();
558        for mut diff in std::mem::replace(&mut diffs, Vec::with_capacity(len)) {
559            let Some(depth) = self.arena.get_depth(diff.idx) else {
560                warn!("{:?} is not in arena. It could be a dangling container that was deleted before the shallow start version.", self.arena.idx_to_id(diff.idx));
561                continue;
562            };
563            let this_depth = depth.get();
564            while this_depth > last_depth {
565                // Clear `to_revive` when we are going to process a new level
566                // so that we can process the revival of the next level
567                let to_create = std::mem::take(&mut to_revive_in_this_layer);
568                to_revive_in_this_layer = std::mem::take(&mut to_revive_in_next_layer);
569                for new in to_create {
570                    let state = self.store.get_or_create_mut(new);
571                    if state.is_state_empty() {
572                        continue;
573                    }
574
575                    let external_diff = state.to_diff(&self.doc);
576                    trigger_on_new_container(
577                        &external_diff,
578                        |cid| {
579                            to_revive_in_this_layer.insert(cid);
580                        },
581                        &self.arena,
582                    );
583
584                    diffs.push(InternalContainerDiff {
585                        idx: new,
586                        bring_back: true,
587                        diff: external_diff.into(),
588                        diff_mode: DiffMode::Checkout,
589                    });
590                }
591
592                last_depth += 1;
593            }
594
595            let idx = diff.idx;
596            let internal_diff = std::mem::take(&mut diff.diff);
597            match &internal_diff {
598                crate::event::DiffVariant::None => {
599                    if is_recording {
600                        let state = self.store.get_or_create_mut(diff.idx);
601                        let extern_diff = state.to_diff(&self.doc);
602                        trigger_on_new_container(
603                            &extern_diff,
604                            |cid| {
605                                to_revive_in_next_layer.insert(cid);
606                            },
607                            &self.arena,
608                        );
609                        diff.diff = extern_diff.into();
610                    }
611                }
612                crate::event::DiffVariant::Internal(_) => {
613                    let cid = self.arena.idx_to_id(idx).unwrap();
614                    info_span!("apply diff on", container_id = ?cid).in_scope(|| {
615                        if self.in_txn {
616                            self.changed_idx_in_txn.insert(idx);
617                        }
618                        let state = self.store.get_or_create_mut(idx);
619                        if is_recording {
620                            // process bring_back before apply
621                            let external_diff =
622                                if diff.bring_back || to_revive_in_this_layer.contains(&idx) {
623                                    state.apply_diff(
624                                        internal_diff.into_internal().unwrap(),
625                                        DiffApplyContext {
626                                            mode: diff.diff_mode,
627                                            doc: &self.doc,
628                                        },
629                                    );
630                                    state.to_diff(&self.doc)
631                                } else {
632                                    state.apply_diff_and_convert(
633                                        internal_diff.into_internal().unwrap(),
634                                        DiffApplyContext {
635                                            mode: diff.diff_mode,
636                                            doc: &self.doc,
637                                        },
638                                    )
639                                };
640                            trigger_on_new_container(
641                                &external_diff,
642                                |cid| {
643                                    to_revive_in_next_layer.insert(cid);
644                                },
645                                &self.arena,
646                            );
647                            diff.diff = external_diff.into();
648                        } else {
649                            state.apply_diff(
650                                internal_diff.into_internal().unwrap(),
651                                DiffApplyContext {
652                                    mode: diff.diff_mode,
653                                    doc: &self.doc,
654                                },
655                            );
656                        }
657                    });
658                }
659                crate::event::DiffVariant::External(_) => unreachable!(),
660            }
661
662            to_revive_in_this_layer.remove(&idx);
663            if !diff.diff.is_empty() {
664                diffs.push(diff);
665            }
666        }
667
668        // Revive the last several layers
669        while !to_revive_in_this_layer.is_empty() || !to_revive_in_next_layer.is_empty() {
670            let to_create = std::mem::take(&mut to_revive_in_this_layer);
671            for new in to_create {
672                let state = self.store.get_or_create_mut(new);
673                if state.is_state_empty() {
674                    continue;
675                }
676
677                let external_diff = state.to_diff(&self.doc);
678                trigger_on_new_container(
679                    &external_diff,
680                    |cid| {
681                        to_revive_in_next_layer.insert(cid);
682                    },
683                    &self.arena,
684                );
685
686                if !external_diff.is_empty() {
687                    diffs.push(InternalContainerDiff {
688                        idx: new,
689                        bring_back: true,
690                        diff: external_diff.into(),
691                        diff_mode: DiffMode::Checkout,
692                    });
693                }
694            }
695
696            to_revive_in_this_layer = std::mem::take(&mut to_revive_in_next_layer);
697        }
698
699        diff.diff = diffs.into();
700        self.frontiers = diff.new_version.clone().into_owned();
701        if self.is_recording() {
702            self.record_diff(diff)
703        }
704    }
705
706    pub fn apply_local_op(&mut self, raw_op: &RawOp, op: &Op) -> LoroResult<()> {
707        // set parent first, `MapContainer` will only be created for TreeID that does not contain
708        self.set_container_parent_by_raw_op(raw_op);
709        let state = self.store.get_or_create_mut(op.container);
710        if self.in_txn {
711            self.changed_idx_in_txn.insert(op.container);
712        }
713        let ret = state.apply_local_op(raw_op, op)?;
714        if !ret.deleted_containers.is_empty() {
715            self.dead_containers_cache.clear_alive();
716        }
717
718        Ok(())
719    }
720
721    pub(crate) fn start_txn(&mut self, origin: InternalString, trigger: EventTriggerKind) {
722        self.pre_txn(origin, trigger);
723        self.in_txn = true;
724    }
725
726    pub(crate) fn abort_txn(&mut self) {
727        self.in_txn = false;
728    }
729
730    pub fn iter_and_decode_all(&mut self) -> impl Iterator<Item = &mut State> {
731        self.store.iter_and_decode_all()
732    }
733
734    pub(crate) fn iter_all_containers_mut(
735        &mut self,
736    ) -> impl Iterator<Item = (&ContainerIdx, &mut ContainerWrapper)> {
737        self.store.iter_all_containers()
738    }
739
740    pub fn does_container_exist(&self, id: &ContainerID) -> bool {
741        // TODO: we may need a better way to handle this in the future when we need to enable fully lazy loading on state
742        self.arena.id_to_idx(id).is_some()
743    }
744
745    pub(crate) fn init_container(
746        &mut self,
747        cid: ContainerID,
748        decode_ctx: StateSnapshotDecodeContext,
749    ) -> LoroResult<()> {
750        let idx = self.arena.register_container(&cid);
751        let state = self.store.get_or_create_mut(idx);
752        state.import_from_snapshot_ops(decode_ctx)
753    }
754
755    pub(crate) fn init_unknown_container(&mut self, cid: ContainerID) {
756        let idx = self.arena.register_container(&cid);
757        self.store.get_or_create_imm(idx);
758    }
759
760    pub(crate) fn commit_txn(&mut self, new_frontiers: Frontiers, diff: Option<InternalDocDiff>) {
761        self.in_txn = false;
762        self.frontiers = new_frontiers;
763        if self.is_recording() {
764            self.record_diff(diff.unwrap());
765        }
766    }
767
768    #[inline]
769    pub(super) fn get_container_mut(&mut self, idx: ContainerIdx) -> Option<&mut State> {
770        self.store.get_container_mut(idx)
771    }
772
773    /// Ensure the container is created and will be encoded in the next `encode` call
774    #[inline]
775    pub(crate) fn ensure_container(&mut self, id: &ContainerID) {
776        self.store.ensure_container(id);
777    }
778
779    /// Ensure all alive containers are created in DocState and will be encoded in the next `encode` call
780    pub(crate) fn ensure_all_alive_containers(&mut self) -> FxHashSet<ContainerID> {
781        // TODO: PERF This can be optimized because we shouldn't need to call get_value for
782        // all the containers every time we export
783        let ans = self.get_all_alive_containers();
784        for id in ans.iter() {
785            self.ensure_container(id);
786        }
787
788        ans
789    }
790
791    pub(crate) fn get_value_by_idx(&mut self, container_idx: ContainerIdx) -> LoroValue {
792        self.store
793            .get_value(container_idx)
794            .unwrap_or_else(|| container_idx.get_type().default_value())
795    }
796
797    /// Set the state of the container with the given container idx.
798    /// This is only used for decode.
799    ///
800    /// # Panic
801    ///
802    /// If the state is not empty.
803    pub(super) fn init_with_states_and_version(
804        &mut self,
805        frontiers: Frontiers,
806        oplog: &OpLog,
807        unknown_containers: Vec<ContainerIdx>,
808        need_to_register_parent: bool,
809    ) {
810        self.pre_txn(Default::default(), EventTriggerKind::Import);
811        if need_to_register_parent {
812            for state in self.store.iter_and_decode_all() {
813                let idx = state.container_idx();
814                let s = state;
815                for child_id in s.get_child_containers() {
816                    let child_idx = self.arena.register_container(&child_id);
817                    self.arena.set_parent(child_idx, Some(idx));
818                }
819            }
820        }
821
822        if !unknown_containers.is_empty() {
823            let mut diff_calc = DiffCalculator::new(false);
824            let stack_vv;
825            let vv = if oplog.frontiers() == &frontiers {
826                oplog.vv()
827            } else {
828                stack_vv = oplog.dag().frontiers_to_vv(&frontiers);
829                stack_vv.as_ref().unwrap()
830            };
831
832            let (unknown_diffs, _diff_mode) = diff_calc.calc_diff_internal(
833                oplog,
834                &Default::default(),
835                &Default::default(),
836                vv,
837                &frontiers,
838                Some(&|idx| !idx.is_unknown() && unknown_containers.contains(&idx)),
839            );
840            self.apply_diff(
841                InternalDocDiff {
842                    origin: Default::default(),
843                    by: EventTriggerKind::Import,
844                    diff: unknown_diffs.into(),
845                    new_version: Cow::Owned(frontiers.clone()),
846                },
847                DiffMode::Checkout,
848            )
849        }
850
851        if self.is_recording() {
852            let diff: Vec<_> = self
853                .store
854                .iter_all_containers()
855                .map(|(&idx, state)| InternalContainerDiff {
856                    idx,
857                    bring_back: false,
858                    diff: state
859                        .get_state_mut(
860                            idx,
861                            ContainerCreationContext {
862                                configure: &self.config,
863                                peer: self.peer.load(Ordering::Relaxed),
864                            },
865                        )
866                        .to_diff(&self.doc)
867                        .into(),
868                    diff_mode: DiffMode::Checkout,
869                })
870                .collect();
871
872            self.record_diff(InternalDocDiff {
873                origin: Default::default(),
874                by: EventTriggerKind::Import,
875                diff: diff.into(),
876                new_version: Cow::Borrowed(&frontiers),
877            });
878        }
879
880        self.frontiers = frontiers;
881    }
882
883    /// id can be a str, ContainerID, or ContainerIdRaw.
884    /// if it's str it will use Root container, which will not be None
885    pub fn get_text<I: Into<ContainerIdRaw>>(
886        &mut self,
887        id: I,
888    ) -> Option<&mut richtext_state::RichtextState> {
889        let idx = self.id_to_idx(id, ContainerType::Text);
890        self.store
891            .get_or_create_mut(idx)
892            .as_richtext_state_mut()
893            .map(|x| &mut **x)
894    }
895
896    /// id can be a str, ContainerID, or ContainerIdRaw.
897    /// if it's str it will use Root container, which will not be None
898    #[allow(unused)]
899    pub(crate) fn get_tree<I: Into<ContainerIdRaw>>(&mut self, id: I) -> Option<&mut TreeState> {
900        let idx = self.id_to_idx(id, ContainerType::Tree);
901        self.store
902            .get_or_create_mut(idx)
903            .as_tree_state_mut()
904            .map(|x| &mut **x)
905    }
906
907    fn id_to_idx<I: Into<ContainerIdRaw>>(&mut self, id: I, kind: ContainerType) -> ContainerIdx {
908        let id: ContainerIdRaw = id.into();
909        let cid;
910        let idx = match id {
911            ContainerIdRaw::Root { name } => {
912                cid = crate::container::ContainerID::Root {
913                    name,
914                    container_type: kind,
915                };
916                Some(self.arena.register_container(&cid))
917            }
918            ContainerIdRaw::Normal { id: _ } => {
919                cid = id.with_type(kind);
920                self.arena.id_to_idx(&cid)
921            }
922        };
923
924        idx.unwrap()
925    }
926
927    #[inline(always)]
928    #[allow(unused)]
929    pub(crate) fn with_state<F, R>(&mut self, idx: ContainerIdx, f: F) -> R
930    where
931        F: FnOnce(&State) -> R,
932    {
933        let depth = self.arena.get_depth(idx).unwrap().get() as usize;
934        let state = self.store.get_or_create_imm(idx);
935        f(state)
936    }
937
938    #[inline(always)]
939    pub(crate) fn with_state_mut<F, R>(&mut self, idx: ContainerIdx, f: F) -> R
940    where
941        F: FnOnce(&mut State) -> R,
942    {
943        let state = self.store.get_or_create_mut(idx);
944        f(state)
945    }
946
947    pub(super) fn is_in_txn(&self) -> bool {
948        self.in_txn
949    }
950
951    pub fn can_import_snapshot(&self) -> bool {
952        !self.in_txn && self.arena.can_import_snapshot() && self.store.can_import_snapshot()
953    }
954
955    pub fn get_value(&self) -> LoroValue {
956        let roots = self.arena.root_containers();
957        let ans: loro_common::LoroMapValue = roots
958            .into_iter()
959            .map(|idx| {
960                let id = self.arena.idx_to_id(idx).unwrap();
961                let ContainerID::Root {
962                    name,
963                    container_type: _,
964                } = &id
965                else {
966                    unreachable!()
967                };
968                (name.to_string(), LoroValue::Container(id))
969            })
970            .collect();
971        LoroValue::Map(ans)
972    }
973
974    pub fn get_deep_value(&mut self) -> LoroValue {
975        let roots = self.arena.root_containers();
976        let mut ans = FxHashMap::with_capacity_and_hasher(roots.len(), Default::default());
977        let binding = self.config.deleted_root_containers.clone();
978        let deleted_root_container = binding.lock().unwrap();
979        let should_hide_empty_root_container = self
980            .config
981            .hide_empty_root_containers
982            .load(Ordering::Relaxed);
983        for root_idx in roots {
984            let id = self.arena.idx_to_id(root_idx).unwrap();
985            match &id {
986                loro_common::ContainerID::Root { name, .. } => {
987                    let v = self.get_container_deep_value(root_idx);
988                    if (should_hide_empty_root_container || deleted_root_container.contains(&id))
989                        && v.is_empty_collection()
990                    {
991                        continue;
992                    }
993
994                    ans.insert(name.to_string(), v);
995                }
996                loro_common::ContainerID::Normal { .. } => {
997                    unreachable!()
998                }
999            }
1000        }
1001
1002        LoroValue::Map(ans.into())
1003    }
1004
1005    pub fn get_deep_value_with_id(&mut self) -> LoroValue {
1006        let roots = self.arena.root_containers();
1007        let mut ans = FxHashMap::with_capacity_and_hasher(roots.len(), Default::default());
1008        for root_idx in roots {
1009            let id = self.arena.idx_to_id(root_idx).unwrap();
1010            match id.clone() {
1011                loro_common::ContainerID::Root { name, .. } => {
1012                    ans.insert(
1013                        name.to_string(),
1014                        self.get_container_deep_value_with_id(root_idx, Some(id)),
1015                    );
1016                }
1017                loro_common::ContainerID::Normal { .. } => {
1018                    unreachable!()
1019                }
1020            }
1021        }
1022
1023        LoroValue::Map(ans.into())
1024    }
1025
1026    pub fn get_all_container_value_flat(&mut self) -> LoroValue {
1027        let mut map = FxHashMap::default();
1028        self.store.iter_and_decode_all().for_each(|c| {
1029            let value = c.get_value();
1030            let cid = self.arena.idx_to_id(c.container_idx()).unwrap().to_string();
1031            map.insert(cid, value);
1032        });
1033
1034        LoroValue::Map(map.into())
1035    }
1036
1037    pub(crate) fn get_container_deep_value_with_id(
1038        &mut self,
1039        container: ContainerIdx,
1040        id: Option<ContainerID>,
1041    ) -> LoroValue {
1042        let id = id.unwrap_or_else(|| self.arena.idx_to_id(container).unwrap());
1043        let Some(state) = self.store.get_container_mut(container) else {
1044            return container.get_type().default_value();
1045        };
1046        let value = state.get_value();
1047        let cid_str = LoroValue::String(format!("idx:{}, id:{}", container.to_index(), id).into());
1048        match value {
1049            LoroValue::Container(_) => unreachable!(),
1050            LoroValue::List(mut list) => {
1051                if container.get_type() == ContainerType::Tree {
1052                    get_meta_value(list.make_mut(), self);
1053                } else {
1054                    if list.iter().all(|x| !x.is_container()) {
1055                        return LoroValue::Map(
1056                            (fx_map!(
1057                                "cid".into() => cid_str,
1058                                "value".into() =>  LoroValue::List(list)
1059                            ))
1060                            .into(),
1061                        );
1062                    }
1063
1064                    let list_mut = list.make_mut();
1065                    for item in list_mut.iter_mut() {
1066                        if item.is_container() {
1067                            let container = item.as_container().unwrap();
1068                            let container_idx = self.arena.register_container(container);
1069                            let value = self.get_container_deep_value_with_id(
1070                                container_idx,
1071                                Some(container.clone()),
1072                            );
1073                            *item = value;
1074                        }
1075                    }
1076                }
1077                LoroValue::Map(
1078                    (fx_map!(
1079                        "cid".into() => cid_str,
1080                        "value".into() => LoroValue::List(list)
1081                    ))
1082                    .into(),
1083                )
1084            }
1085            LoroValue::Map(mut map) => {
1086                let map_mut = map.make_mut();
1087                for (_key, value) in map_mut.iter_mut() {
1088                    if value.is_container() {
1089                        let container = value.as_container().unwrap();
1090                        let container_idx = self.arena.register_container(container);
1091                        let new_value = self.get_container_deep_value_with_id(
1092                            container_idx,
1093                            Some(container.clone()),
1094                        );
1095                        *value = new_value;
1096                    }
1097                }
1098
1099                LoroValue::Map(
1100                    (fx_map!(
1101                        "cid".into() => cid_str,
1102                        "value".into() => LoroValue::Map(map)
1103                    ))
1104                    .into(),
1105                )
1106            }
1107            _ => LoroValue::Map(
1108                (fx_map!(
1109                    "cid".into() => cid_str,
1110                    "value".into() => value
1111                ))
1112                .into(),
1113            ),
1114        }
1115    }
1116
1117    pub fn get_container_deep_value(&mut self, container: ContainerIdx) -> LoroValue {
1118        let Some(value) = self.store.get_value(container) else {
1119            return container.get_type().default_value();
1120        };
1121        match value {
1122            LoroValue::Container(_) => unreachable!(),
1123            LoroValue::List(mut list) => {
1124                if container.get_type() == ContainerType::Tree {
1125                    // Each tree node has an associated map container to represent
1126                    // the metadata of this node. When the user get the deep value,
1127                    // we need to add a field named `meta` to the tree node,
1128                    // whose value is deep value of map container.
1129                    get_meta_value(list.make_mut(), self);
1130                } else {
1131                    if list.iter().all(|x| !x.is_container()) {
1132                        return LoroValue::List(list);
1133                    }
1134
1135                    let list_mut = list.make_mut();
1136                    for item in list_mut.iter_mut() {
1137                        if item.is_container() {
1138                            let container = item.as_container().unwrap();
1139                            let container_idx = self.arena.register_container(container);
1140                            let value = self.get_container_deep_value(container_idx);
1141                            *item = value;
1142                        }
1143                    }
1144                }
1145                LoroValue::List(list)
1146            }
1147            LoroValue::Map(mut map) => {
1148                if map.iter().all(|x| !x.1.is_container()) {
1149                    return LoroValue::Map(map);
1150                }
1151
1152                let map_mut = map.make_mut();
1153                for (_key, value) in map_mut.iter_mut() {
1154                    if value.is_container() {
1155                        let container = value.as_container().unwrap();
1156                        let container_idx = self.arena.register_container(container);
1157                        let new_value = self.get_container_deep_value(container_idx);
1158                        *value = new_value;
1159                    }
1160                }
1161                LoroValue::Map(map)
1162            }
1163            _ => value,
1164        }
1165    }
1166
1167    pub(crate) fn get_all_alive_containers(&mut self) -> FxHashSet<ContainerID> {
1168        let mut ans = FxHashSet::default();
1169        let mut to_visit = self
1170            .arena
1171            .root_containers()
1172            .iter()
1173            .map(|x| self.arena.get_container_id(*x).unwrap())
1174            .collect_vec();
1175
1176        while let Some(id) = to_visit.pop() {
1177            self.get_alive_children_of(&id, &mut to_visit);
1178            ans.insert(id);
1179        }
1180
1181        ans
1182    }
1183
1184    pub(crate) fn get_alive_children_of(&mut self, id: &ContainerID, ans: &mut Vec<ContainerID>) {
1185        let idx = self.arena.register_container(id);
1186        let Some(value) = self.store.get_value(idx) else {
1187            return;
1188        };
1189
1190        match value {
1191            LoroValue::Container(_) => unreachable!(),
1192            LoroValue::List(list) => {
1193                if idx.get_type() == ContainerType::Tree {
1194                    // Each tree node has an associated map container to represent
1195                    // the metadata of this node. When the user get the deep value,
1196                    // we need to add a field named `meta` to the tree node,
1197                    // whose value is deep value of map container.
1198                    let mut list = list.unwrap();
1199                    while let Some(node) = list.pop() {
1200                        let map = node.as_map().unwrap();
1201                        let meta = map.get("meta").unwrap();
1202                        let id = meta.as_container().unwrap();
1203                        ans.push(id.clone());
1204                        let children = map.get("children").unwrap();
1205                        let children = children.as_list().unwrap();
1206                        for child in children.iter() {
1207                            list.push(child.clone());
1208                        }
1209                    }
1210                } else {
1211                    for item in list.iter() {
1212                        if let LoroValue::Container(id) = item {
1213                            ans.push(id.clone());
1214                        }
1215                    }
1216                }
1217            }
1218            LoroValue::Map(map) => {
1219                for (_key, value) in map.iter() {
1220                    if let LoroValue::Container(id) = value {
1221                        ans.push(id.clone());
1222                    }
1223                }
1224            }
1225            _ => {}
1226        }
1227    }
1228
1229    // Because we need to calculate path based on [DocState], so we cannot extract
1230    // the event recorder to a separate module.
1231    fn diffs_to_event(&mut self, diffs: Vec<InternalDocDiff<'_>>, from: Frontiers) -> DocDiff {
1232        if diffs.is_empty() {
1233            panic!("diffs is empty");
1234        }
1235
1236        let triggered_by = diffs[0].by;
1237        debug_assert!(diffs.iter().all(|x| x.by == triggered_by));
1238        let mut containers = FxHashMap::default();
1239        let to = (*diffs.last().unwrap().new_version).to_owned();
1240        let origin = diffs[0].origin.clone();
1241        for diff in diffs {
1242            #[allow(clippy::unnecessary_to_owned)]
1243            for container_diff in diff.diff.into_owned() {
1244                let Some((last_container_diff, _)) = containers.get_mut(&container_diff.idx) else {
1245                    if let Some(path) = self.get_path(container_diff.idx) {
1246                        containers.insert(container_diff.idx, (container_diff.diff, path));
1247                    } else {
1248                        // if we cannot find the path to the container, the container must be overwritten afterwards.
1249                        // So we can ignore the diff from it.
1250                        loro_common::warn!(
1251                            "⚠️ WARNING: ignore event because cannot find its path {:#?} container id:{}",
1252                            &container_diff,
1253                            self.arena.idx_to_id(container_diff.idx).unwrap()
1254                        );
1255                    }
1256
1257                    continue;
1258                };
1259                // TODO: PERF avoid this clone
1260                *last_container_diff = last_container_diff
1261                    .clone()
1262                    .compose(container_diff.diff)
1263                    .unwrap();
1264            }
1265        }
1266        let mut diff: Vec<_> = containers
1267            .into_iter()
1268            .map(|(container, (diff, path))| {
1269                let idx = container;
1270                let id = self.arena.get_container_id(idx).unwrap();
1271                let is_unknown = id.is_unknown();
1272
1273                ContainerDiff {
1274                    id,
1275                    idx,
1276                    diff: diff.into_external().unwrap(),
1277                    is_unknown,
1278                    path,
1279                }
1280            })
1281            .collect();
1282
1283        // Sort by path length, so caller can apply the diff from the root to the leaf.
1284        // Otherwise, the caller may use a wrong path to apply the diff.
1285
1286        diff.sort_by_key(|x| {
1287            (
1288                x.path.len(),
1289                match &x.id {
1290                    ContainerID::Root { .. } => 0,
1291                    ContainerID::Normal { counter, .. } => *counter + 1,
1292                },
1293            )
1294        });
1295        DocDiff {
1296            from,
1297            to,
1298            origin,
1299            by: triggered_by,
1300            diff,
1301        }
1302    }
1303
1304    pub(crate) fn get_reachable(&mut self, id: &ContainerID) -> bool {
1305        if matches!(id, ContainerID::Root { .. }) {
1306            return true;
1307        }
1308
1309        let Some(mut idx) = self.arena.id_to_idx(id) else {
1310            return false;
1311        };
1312        loop {
1313            let id = self.arena.idx_to_id(idx).unwrap();
1314            if let Some(parent_idx) = self.arena.get_parent(idx) {
1315                let Some(parent_state) = self.store.get_container_mut(parent_idx) else {
1316                    return false;
1317                };
1318                if !parent_state.contains_child(&id) {
1319                    return false;
1320                }
1321                idx = parent_idx;
1322            } else {
1323                if id.is_root() {
1324                    return true;
1325                }
1326
1327                return false;
1328            }
1329        }
1330    }
1331
1332    // the container may be override, so it may return None
1333    pub(super) fn get_path(&mut self, idx: ContainerIdx) -> Option<Vec<(ContainerID, Index)>> {
1334        let mut ans = Vec::new();
1335        let mut idx = idx;
1336        loop {
1337            let id = self.arena.idx_to_id(idx).unwrap();
1338            if let Some(parent_idx) = self.arena.get_parent(idx) {
1339                let parent_state = self.store.get_container_mut(parent_idx)?;
1340                let Some(prop) = parent_state.get_child_index(&id) else {
1341                    tracing::info!("Missing in parent children");
1342                    return None;
1343                };
1344                ans.push((id, prop));
1345                idx = parent_idx;
1346            } else {
1347                // this container may be deleted
1348                let Ok(prop) = id.clone().into_root() else {
1349                    let id = format!("{}", &id);
1350                    tracing::info!(?id, "Missing parent - container is deleted");
1351                    return None;
1352                };
1353                ans.push((id, Index::Key(prop.0)));
1354                break;
1355            }
1356        }
1357
1358        ans.reverse();
1359
1360        Some(ans)
1361    }
1362
1363    pub(crate) fn check_before_decode_snapshot(&self) -> LoroResult<()> {
1364        if self.is_in_txn() {
1365            return Err(LoroError::DecodeError(
1366                "State is in txn".to_string().into_boxed_str(),
1367            ));
1368        }
1369
1370        if !self.can_import_snapshot() {
1371            return Err(LoroError::DecodeError(
1372                "State is not empty, cannot import snapshot directly"
1373                    .to_string()
1374                    .into_boxed_str(),
1375            ));
1376        }
1377
1378        Ok(())
1379    }
1380
1381    /// Check whether two [DocState]s are the same. Panic if not.
1382    ///
1383    /// Compared to check equality on `get_deep_value`, this function also checks the equality on richtext
1384    /// styles and states that are not reachable from the root.
1385    ///
1386    /// This is only used for test.
1387    pub(crate) fn check_is_the_same(&mut self, other: &mut Self) {
1388        fn get_entries_for_state(
1389            arena: &SharedArena,
1390            state: &mut State,
1391        ) -> Option<(ContainerID, (ContainerIdx, LoroValue))> {
1392            if state.is_state_empty() {
1393                return None;
1394            }
1395
1396            let id = arena.idx_to_id(state.container_idx()).unwrap();
1397            let value = match state {
1398                State::RichtextState(s) => s.get_richtext_value(),
1399                _ => state.get_value(),
1400            };
1401            if match &value {
1402                LoroValue::List(l) => l.is_empty(),
1403                LoroValue::Map(m) => m.is_empty(),
1404                _ => false,
1405            } {
1406                return None;
1407            }
1408            #[cfg(feature = "counter")]
1409            if id.container_type() == ContainerType::Counter {
1410                if let LoroValue::Double(c) = value {
1411                    if c.abs() < f64::EPSILON {
1412                        return None;
1413                    }
1414                }
1415            }
1416
1417            Some((id, (state.container_idx(), value)))
1418        }
1419
1420        let self_id_to_states: FxHashMap<ContainerID, (ContainerIdx, LoroValue)> = self
1421            .store
1422            .iter_and_decode_all()
1423            .filter_map(|state: &mut State| {
1424                let arena = &self.arena;
1425                get_entries_for_state(arena, state)
1426            })
1427            .collect();
1428        let mut other_id_to_states: FxHashMap<ContainerID, (ContainerIdx, LoroValue)> = other
1429            .store
1430            .iter_and_decode_all()
1431            .filter_map(|state: &mut State| {
1432                let arena = &other.arena;
1433                get_entries_for_state(arena, state)
1434            })
1435            .collect();
1436        for (id, (idx, this_value)) in self_id_to_states {
1437            let (_, other_value) = match other_id_to_states.remove(&id) {
1438                Some(x) => x,
1439                None => {
1440                    panic!(
1441                        "id: {:?}, path: {:?} is missing, value={:?}",
1442                        id,
1443                        self.get_path(idx),
1444                        &this_value
1445                    );
1446                }
1447            };
1448
1449            pretty_assertions::assert_eq!(
1450                this_value,
1451                other_value,
1452                "[self!=other] id: {:?}, path: {:?}",
1453                id,
1454                self.get_path(idx)
1455            );
1456        }
1457
1458        if !other_id_to_states.is_empty() {
1459            panic!("other has more states {:#?}", &other_id_to_states);
1460        }
1461    }
1462
1463    pub fn log_estimated_size(&self) {
1464        let state_entries_size =
1465            self.store.len() * (std::mem::size_of::<State>() + std::mem::size_of::<ContainerIdx>());
1466        let mut state_size_sum = 0;
1467        state_size_sum += self.store.estimate_size();
1468
1469        eprintln!(
1470            "ContainerNum: {}\nEstimated state size: \nEntries: {} \nSum: {}",
1471            self.store.len(),
1472            state_entries_size,
1473            state_size_sum
1474        );
1475    }
1476
1477    pub fn create_state(&self, idx: ContainerIdx) -> State {
1478        let config = &self.config;
1479        let peer = self.peer.load(std::sync::atomic::Ordering::Relaxed);
1480        create_state_(idx, config, peer)
1481    }
1482
1483    pub fn create_unknown_state(&self, idx: ContainerIdx) -> State {
1484        State::UnknownState(UnknownState::new(idx))
1485    }
1486
1487    pub fn get_relative_position(&mut self, pos: &Cursor, use_event_index: bool) -> Option<usize> {
1488        let idx = self.arena.register_container(&pos.container);
1489        let state = self.store.get_container_mut(idx)?;
1490        if let Some(id) = pos.id {
1491            match state {
1492                State::ListState(s) => s.get_index_of_id(id),
1493                State::RichtextState(s) => s.get_text_index_of_id(id, use_event_index),
1494                State::MovableListState(s) => s.get_index_of_id(id),
1495                State::MapState(_) | State::TreeState(_) | State::UnknownState(_) => unreachable!(),
1496                #[cfg(feature = "counter")]
1497                State::CounterState(_) => unreachable!(),
1498            }
1499        } else {
1500            if matches!(pos.side, crate::cursor::Side::Left) {
1501                return Some(0);
1502            }
1503
1504            match state {
1505                State::ListState(s) => Some(s.len()),
1506                State::RichtextState(s) => Some(if use_event_index {
1507                    s.len_event()
1508                } else {
1509                    s.len_unicode()
1510                }),
1511                State::MovableListState(s) => Some(s.len()),
1512                State::MapState(_) | State::TreeState(_) | State::UnknownState(_) => unreachable!(),
1513                #[cfg(feature = "counter")]
1514                State::CounterState(_) => unreachable!(),
1515            }
1516        }
1517    }
1518
1519    pub fn get_value_by_path(&mut self, path: &[Index]) -> Option<LoroValue> {
1520        if path.is_empty() {
1521            return None;
1522        }
1523
1524        enum CurContainer {
1525            Container(ContainerIdx),
1526            TreeNode {
1527                tree: ContainerIdx,
1528                node: Option<TreeID>,
1529            },
1530        }
1531
1532        let mut state_idx = {
1533            let root_index = path[0].as_key()?;
1534            CurContainer::Container(self.arena.get_root_container_idx_by_key(root_index)?)
1535        };
1536
1537        if path.len() == 1 {
1538            if let CurContainer::Container(c) = state_idx {
1539                let cid = self.arena.idx_to_id(c)?;
1540                return Some(LoroValue::Container(cid));
1541            }
1542        }
1543
1544        let mut i = 1;
1545        while i < path.len() - 1 {
1546            let index = &path[i];
1547            match state_idx {
1548                CurContainer::Container(idx) => {
1549                    let parent_state = self.store.get_container_mut(idx)?;
1550                    match parent_state {
1551                        State::ListState(l) => {
1552                            let Some(LoroValue::Container(c)) = l.get(*index.as_seq()?) else {
1553                                return None;
1554                            };
1555                            state_idx = CurContainer::Container(self.arena.register_container(c));
1556                        }
1557                        State::MovableListState(l) => {
1558                            let Some(LoroValue::Container(c)) =
1559                                l.get(*index.as_seq()?, IndexType::ForUser)
1560                            else {
1561                                return None;
1562                            };
1563                            state_idx = CurContainer::Container(self.arena.register_container(c));
1564                        }
1565                        State::MapState(m) => {
1566                            let Some(LoroValue::Container(c)) = m.get(index.as_key()?) else {
1567                                return None;
1568                            };
1569                            state_idx = CurContainer::Container(self.arena.register_container(c));
1570                        }
1571                        State::RichtextState(_) => return None,
1572                        State::TreeState(_) => {
1573                            state_idx = CurContainer::TreeNode {
1574                                tree: idx,
1575                                node: None,
1576                            };
1577                            continue;
1578                        }
1579                        #[cfg(feature = "counter")]
1580                        State::CounterState(_) => return None,
1581                        State::UnknownState(_) => unreachable!(),
1582                    }
1583                }
1584                CurContainer::TreeNode { tree, node } => match index {
1585                    Index::Key(internal_string) => {
1586                        let node = node?;
1587                        let idx = self
1588                            .arena
1589                            .register_container(&node.associated_meta_container());
1590                        let map = self.store.get_container(idx)?;
1591                        let Some(LoroValue::Container(c)) =
1592                            map.as_map_state().unwrap().get(internal_string)
1593                        else {
1594                            return None;
1595                        };
1596
1597                        state_idx = CurContainer::Container(self.arena.register_container(c));
1598                    }
1599                    Index::Seq(i) => {
1600                        let tree_state =
1601                            self.store.get_container_mut(tree)?.as_tree_state().unwrap();
1602                        let parent: TreeParentId = if let Some(node) = node {
1603                            node.into()
1604                        } else {
1605                            TreeParentId::Root
1606                        };
1607                        let child = tree_state.get_children(&parent)?.nth(*i)?;
1608                        state_idx = CurContainer::TreeNode {
1609                            tree,
1610                            node: Some(child),
1611                        };
1612                    }
1613                    Index::Node(tree_id) => {
1614                        let tree_state =
1615                            self.store.get_container_mut(tree)?.as_tree_state().unwrap();
1616                        if tree_state.parent(tree_id).is_some() {
1617                            state_idx = CurContainer::TreeNode {
1618                                tree,
1619                                node: Some(*tree_id),
1620                            }
1621                        } else {
1622                            return None;
1623                        }
1624                    }
1625                },
1626            }
1627            i += 1;
1628        }
1629
1630        let parent_idx = match state_idx {
1631            CurContainer::Container(container_idx) => container_idx,
1632            CurContainer::TreeNode { tree, node } => {
1633                if let Some(node) = node {
1634                    self.arena
1635                        .register_container(&node.associated_meta_container())
1636                } else {
1637                    tree
1638                }
1639            }
1640        };
1641
1642        let parent_state = self.store.get_or_create_mut(parent_idx);
1643        let index = path.last().unwrap();
1644        let value: LoroValue = match parent_state {
1645            State::ListState(l) => l.get(*index.as_seq()?).cloned()?,
1646            State::MovableListState(l) => l.get(*index.as_seq()?, IndexType::ForUser).cloned()?,
1647            State::MapState(m) => {
1648                if let Some(key) = index.as_key() {
1649                    m.get(key).cloned()?
1650                } else if let CurContainer::TreeNode { tree, node } = state_idx {
1651                    match index {
1652                        Index::Seq(index) => {
1653                            let tree_state =
1654                                self.store.get_container_mut(tree)?.as_tree_state().unwrap();
1655                            let parent: TreeParentId = if let Some(node) = node {
1656                                node.into()
1657                            } else {
1658                                TreeParentId::Root
1659                            };
1660                            let child = tree_state.get_children(&parent)?.nth(*index)?;
1661                            child.associated_meta_container().into()
1662                        }
1663                        Index::Node(id) => id.associated_meta_container().into(),
1664                        _ => return None,
1665                    }
1666                } else {
1667                    return None;
1668                }
1669            }
1670            State::RichtextState(s) => {
1671                let s = s.to_string_mut();
1672                s.chars()
1673                    .nth(*index.as_seq()?)
1674                    .map(|c| c.to_string().into())?
1675            }
1676            State::TreeState(_) => {
1677                let id = index.as_node()?;
1678                let cid = id.associated_meta_container();
1679                cid.into()
1680            }
1681            #[cfg(feature = "counter")]
1682            State::CounterState(_) => unreachable!(),
1683            State::UnknownState(_) => unreachable!(),
1684        };
1685
1686        Some(value)
1687    }
1688
1689    pub(crate) fn shallow_root_store(&self) -> Option<&Arc<GcStore>> {
1690        self.store.shallow_root_store()
1691    }
1692}
1693
1694fn create_state_(idx: ContainerIdx, config: &Configure, peer: u64) -> State {
1695    match idx.get_type() {
1696        ContainerType::Map => State::MapState(Box::new(MapState::new(idx))),
1697        ContainerType::List => State::ListState(Box::new(ListState::new(idx))),
1698        ContainerType::Text => State::RichtextState(Box::new(RichtextState::new(
1699            idx,
1700            config.text_style_config.clone(),
1701        ))),
1702        ContainerType::Tree => State::TreeState(Box::new(TreeState::new(idx, peer))),
1703        ContainerType::MovableList => State::MovableListState(Box::new(MovableListState::new(idx))),
1704        #[cfg(feature = "counter")]
1705        ContainerType::Counter => {
1706            State::CounterState(Box::new(counter_state::CounterState::new(idx)))
1707        }
1708        ContainerType::Unknown(_) => State::UnknownState(UnknownState::new(idx)),
1709    }
1710}
1711
1712fn trigger_on_new_container(
1713    state_diff: &Diff,
1714    mut listener: impl FnMut(ContainerIdx),
1715    arena: &SharedArena,
1716) {
1717    match state_diff {
1718        Diff::List(list) => {
1719            for delta in list.iter() {
1720                if let DeltaItem::Replace {
1721                    value,
1722                    attr,
1723                    delete: _,
1724                } = delta
1725                {
1726                    if attr.from_move {
1727                        continue;
1728                    }
1729
1730                    for v in value.iter() {
1731                        if let ValueOrHandler::Handler(h) = v {
1732                            let idx = h.container_idx();
1733                            listener(idx);
1734                        }
1735                    }
1736                }
1737            }
1738        }
1739        Diff::Map(map) => {
1740            for (_, v) in map.updated.iter() {
1741                if let Some(ValueOrHandler::Handler(h)) = &v.value {
1742                    let idx = h.container_idx();
1743                    listener(idx);
1744                }
1745            }
1746        }
1747        Diff::Tree(tree) => {
1748            for item in tree.iter() {
1749                if matches!(item.action, TreeExternalDiff::Create { .. }) {
1750                    let id = item.target.associated_meta_container();
1751                    listener(arena.id_to_idx(&id).unwrap());
1752                }
1753            }
1754        }
1755        _ => {}
1756    };
1757}
1758
1759#[derive(Default, Clone)]
1760struct EventRecorder {
1761    recording_diff: bool,
1762    // A batch of diffs will be converted to a event when
1763    // they cannot be merged with the next diff.
1764    diffs: Vec<InternalDocDiff<'static>>,
1765    events: Vec<DocDiff>,
1766    diff_start_version: Option<Frontiers>,
1767}
1768
1769impl EventRecorder {
1770    #[allow(unused)]
1771    pub fn new() -> Self {
1772        Self::default()
1773    }
1774}
1775
1776#[test]
1777fn test_size() {
1778    println!("Size of State = {}", std::mem::size_of::<State>());
1779    println!("Size of MapState = {}", std::mem::size_of::<MapState>());
1780    println!("Size of ListState = {}", std::mem::size_of::<ListState>());
1781    println!(
1782        "Size of TextState = {}",
1783        std::mem::size_of::<RichtextState>()
1784    );
1785    println!("Size of TreeState = {}", std::mem::size_of::<TreeState>());
1786}