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