sui_adapter_v2/
temporary_store.rs

1// Copyright (c) Mysten Labs, Inc.
2// SPDX-License-Identifier: Apache-2.0
3
4use crate::gas_charger::GasCharger;
5use parking_lot::RwLock;
6use std::collections::{BTreeMap, BTreeSet, HashSet};
7use sui_protocol_config::ProtocolConfig;
8use sui_types::base_types::VersionDigest;
9use sui_types::committee::EpochId;
10use sui_types::digests::ObjectDigest;
11use sui_types::effects::{TransactionEffects, TransactionEvents};
12use sui_types::execution::{
13    DynamicallyLoadedObjectMetadata, ExecutionResults, ExecutionResultsV2, SharedInput,
14};
15use sui_types::execution_status::ExecutionStatus;
16use sui_types::inner_temporary_store::InnerTemporaryStore;
17use sui_types::layout_resolver::LayoutResolver;
18use sui_types::storage::{BackingStore, DenyListResult, PackageObject};
19use sui_types::sui_system_state::{get_sui_system_state_wrapper, AdvanceEpochParams};
20use sui_types::{
21    base_types::{ObjectID, ObjectRef, SequenceNumber, SuiAddress, TransactionDigest},
22    effects::EffectsObjectChange,
23    error::{ExecutionError, SuiResult},
24    gas::GasCostSummary,
25    object::Object,
26    object::Owner,
27    storage::{BackingPackageStore, ChildObjectResolver, ParentSync, Storage},
28    transaction::InputObjects,
29    TypeTag,
30};
31use sui_types::{is_system_package, SUI_SYSTEM_STATE_OBJECT_ID};
32
33pub struct TemporaryStore<'backing> {
34    // The backing store for retrieving Move packages onchain.
35    // When executing a Move call, the dependent packages are not going to be
36    // in the input objects. They will be fetched from the backing store.
37    // Also used for fetching the backing parent_sync to get the last known version for wrapped
38    // objects
39    store: &'backing dyn BackingStore,
40    tx_digest: TransactionDigest,
41    input_objects: BTreeMap<ObjectID, Object>,
42    deleted_consensus_objects: BTreeMap<ObjectID, SequenceNumber>,
43    /// The version to assign to all objects written by the transaction using this store.
44    lamport_timestamp: SequenceNumber,
45    mutable_input_refs: BTreeMap<ObjectID, (VersionDigest, Owner)>, // Inputs that are mutable
46    execution_results: ExecutionResultsV2,
47    /// Objects that were loaded during execution (dynamic fields + received objects).
48    loaded_runtime_objects: BTreeMap<ObjectID, DynamicallyLoadedObjectMetadata>,
49    /// A map from wrapped object to its container. Used during expensive invariant checks.
50    wrapped_object_containers: BTreeMap<ObjectID, ObjectID>,
51    protocol_config: &'backing ProtocolConfig,
52
53    /// Every package that was loaded from DB store during execution.
54    /// These packages were not previously loaded into the temporary store.
55    runtime_packages_loaded_from_db: RwLock<BTreeMap<ObjectID, PackageObject>>,
56
57    /// The set of objects that we may receive during execution. Not guaranteed to receive all, or
58    /// any of the objects referenced in this set.
59    receiving_objects: Vec<ObjectRef>,
60}
61
62impl<'backing> TemporaryStore<'backing> {
63    /// Creates a new store associated with an authority store, and populates it with
64    /// initial objects.
65    pub fn new(
66        store: &'backing dyn BackingStore,
67        input_objects: InputObjects,
68        receiving_objects: Vec<ObjectRef>,
69        tx_digest: TransactionDigest,
70        protocol_config: &'backing ProtocolConfig,
71    ) -> Self {
72        let mutable_input_refs = input_objects.exclusive_mutable_inputs();
73        let lamport_timestamp = input_objects.lamport_timestamp(&receiving_objects);
74        let deleted_consensus_objects = input_objects.consensus_stream_ended_objects();
75        let objects = input_objects.into_object_map();
76        #[cfg(debug_assertions)]
77        {
78            // Ensure that input objects and receiving objects must not overlap.
79            assert!(objects
80                .keys()
81                .collect::<HashSet<_>>()
82                .intersection(
83                    &receiving_objects
84                        .iter()
85                        .map(|oref| &oref.0)
86                        .collect::<HashSet<_>>()
87                )
88                .next()
89                .is_none());
90        }
91        Self {
92            store,
93            tx_digest,
94            input_objects: objects,
95            deleted_consensus_objects,
96            lamport_timestamp,
97            mutable_input_refs,
98            execution_results: ExecutionResultsV2::default(),
99            protocol_config,
100            loaded_runtime_objects: BTreeMap::new(),
101            wrapped_object_containers: BTreeMap::new(),
102            runtime_packages_loaded_from_db: RwLock::new(BTreeMap::new()),
103            receiving_objects,
104        }
105    }
106
107    // Helpers to access private fields
108    pub fn objects(&self) -> &BTreeMap<ObjectID, Object> {
109        &self.input_objects
110    }
111
112    pub fn update_object_version_and_prev_tx(&mut self) {
113        self.execution_results.update_version_and_previous_tx(
114            self.lamport_timestamp,
115            self.tx_digest,
116            &self.input_objects,
117            false,
118        );
119
120        #[cfg(debug_assertions)]
121        {
122            self.check_invariants();
123        }
124    }
125
126    /// Break up the structure and return its internal stores (objects, active_inputs, written, deleted)
127    pub fn into_inner(self) -> InnerTemporaryStore {
128        let results = self.execution_results;
129        InnerTemporaryStore {
130            input_objects: self.input_objects,
131            mutable_inputs: self.mutable_input_refs,
132            stream_ended_consensus_objects: self.deleted_consensus_objects,
133            written: results.written_objects,
134            events: TransactionEvents {
135                data: results.user_events,
136            },
137            // no accumulator events for v2
138            accumulator_events: vec![],
139            loaded_runtime_objects: self.loaded_runtime_objects,
140            runtime_packages_loaded_from_db: self.runtime_packages_loaded_from_db.into_inner(),
141            lamport_version: self.lamport_timestamp,
142            binary_config: self.protocol_config.binary_config(None),
143        }
144    }
145
146    /// For every object from active_inputs (i.e. all mutable objects), if they are not
147    /// mutated during the transaction execution, force mutating them by incrementing the
148    /// sequence number. This is required to achieve safety.
149    pub(crate) fn ensure_active_inputs_mutated(&mut self) {
150        let mut to_be_updated = vec![];
151        for id in self.mutable_input_refs.keys() {
152            if !self.execution_results.modified_objects.contains(id) {
153                // We cannot update here but have to push to `to_be_updated` and update later
154                // because the for loop is holding a reference to `self`, and calling
155                // `self.mutate_input_object` requires a mutable reference to `self`.
156                to_be_updated.push(self.input_objects[id].clone());
157            }
158        }
159        for object in to_be_updated {
160            // The object must be mutated as it was present in the input objects
161            self.mutate_input_object(object.clone());
162        }
163    }
164
165    fn get_object_changes(&self) -> BTreeMap<ObjectID, EffectsObjectChange> {
166        let results = &self.execution_results;
167        let all_ids = results
168            .created_object_ids
169            .iter()
170            .chain(&results.deleted_object_ids)
171            .chain(&results.modified_objects)
172            .chain(results.written_objects.keys())
173            .collect::<BTreeSet<_>>();
174        all_ids
175            .into_iter()
176            .map(|id| {
177                (
178                    *id,
179                    EffectsObjectChange::new(
180                        self.get_object_modified_at(id)
181                            .map(|metadata| ((metadata.version, metadata.digest), metadata.owner)),
182                        results.written_objects.get(id),
183                        results.created_object_ids.contains(id),
184                        results.deleted_object_ids.contains(id),
185                    ),
186                )
187            })
188            .collect()
189    }
190
191    pub fn into_effects(
192        mut self,
193        shared_object_refs: Vec<SharedInput>,
194        transaction_digest: &TransactionDigest,
195        mut transaction_dependencies: BTreeSet<TransactionDigest>,
196        gas_cost_summary: GasCostSummary,
197        status: ExecutionStatus,
198        gas_charger: &mut GasCharger,
199        epoch: EpochId,
200    ) -> (InnerTemporaryStore, TransactionEffects) {
201        self.update_object_version_and_prev_tx();
202
203        // Regardless of execution status (including aborts), we insert the previous transaction
204        // for any successfully received objects during the transaction.
205        for (id, expected_version, expected_digest) in &self.receiving_objects {
206            // If the receiving object is in the loaded runtime objects, then that means that it
207            // was actually successfully loaded (so existed, and there was authenticated mutable
208            // access to it). So we insert the previous transaction as a dependency.
209            if let Some(obj_meta) = self.loaded_runtime_objects.get(id) {
210                // Check that the expected version, digest, and owner match the loaded version,
211                // digest, and owner. If they don't then don't register a dependency.
212                // This is because this could be "spoofed" by loading a dynamic object field.
213                let loaded_via_receive = obj_meta.version == *expected_version
214                    && obj_meta.digest == *expected_digest
215                    && obj_meta.owner.is_address_owned();
216                if loaded_via_receive {
217                    transaction_dependencies.insert(obj_meta.previous_transaction);
218                }
219            }
220        }
221
222        if self.protocol_config.enable_effects_v2() {
223            self.into_effects_v2(
224                shared_object_refs,
225                transaction_digest,
226                transaction_dependencies,
227                gas_cost_summary,
228                status,
229                gas_charger,
230                epoch,
231            )
232        } else {
233            let shared_object_refs = shared_object_refs
234                .into_iter()
235                .map(|shared_input| match shared_input {
236                    SharedInput::Existing(oref) => oref,
237                    SharedInput::ConsensusStreamEnded(_) => {
238                        unreachable!("Shared object deletion not supported in effects v1")
239                    }
240                    SharedInput::Cancelled(_) => {
241                        unreachable!("Per object congestion control not supported in effects v1.")
242                    }
243                })
244                .collect();
245            self.into_effects_v1(
246                shared_object_refs,
247                transaction_digest,
248                transaction_dependencies,
249                gas_cost_summary,
250                status,
251                gas_charger,
252                epoch,
253            )
254        }
255    }
256
257    fn into_effects_v1(
258        self,
259        shared_object_refs: Vec<ObjectRef>,
260        transaction_digest: &TransactionDigest,
261        transaction_dependencies: BTreeSet<TransactionDigest>,
262        gas_cost_summary: GasCostSummary,
263        status: ExecutionStatus,
264        gas_charger: &mut GasCharger,
265        epoch: EpochId,
266    ) -> (InnerTemporaryStore, TransactionEffects) {
267        let updated_gas_object_info = if let Some(coin_id) = gas_charger.gas_coin() {
268            let object = &self.execution_results.written_objects[&coin_id];
269            (object.compute_object_reference(), object.owner.clone())
270        } else {
271            (
272                (ObjectID::ZERO, SequenceNumber::default(), ObjectDigest::MIN),
273                Owner::AddressOwner(SuiAddress::default()),
274            )
275        };
276        let lampot_version = self.lamport_timestamp;
277
278        let mut created = vec![];
279        let mut mutated = vec![];
280        let mut unwrapped = vec![];
281        let mut deleted = vec![];
282        let mut unwrapped_then_deleted = vec![];
283        let mut wrapped = vec![];
284        // It is important that we constructs `modified_at_versions` and `deleted_at_versions`
285        // separately, and merge them latter to achieve the exact same order as in v1.
286        let mut modified_at_versions = vec![];
287        let mut deleted_at_versions = vec![];
288        self.execution_results
289            .written_objects
290            .iter()
291            .for_each(|(id, object)| {
292                let object_ref = object.compute_object_reference();
293                let owner = object.owner.clone();
294                if let Some(old_object_meta) = self.get_object_modified_at(id) {
295                    modified_at_versions.push((*id, old_object_meta.version));
296                    mutated.push((object_ref, owner));
297                } else if self.execution_results.created_object_ids.contains(id) {
298                    created.push((object_ref, owner));
299                } else {
300                    unwrapped.push((object_ref, owner));
301                }
302            });
303        self.execution_results
304            .modified_objects
305            .iter()
306            .filter(|id| !self.execution_results.written_objects.contains_key(id))
307            .for_each(|id| {
308                let old_object_meta = self.get_object_modified_at(id).unwrap();
309                deleted_at_versions.push((*id, old_object_meta.version));
310                if self.execution_results.deleted_object_ids.contains(id) {
311                    deleted.push((*id, lampot_version, ObjectDigest::OBJECT_DIGEST_DELETED));
312                } else {
313                    wrapped.push((*id, lampot_version, ObjectDigest::OBJECT_DIGEST_WRAPPED));
314                }
315            });
316        self.execution_results
317            .deleted_object_ids
318            .iter()
319            .filter(|id| !self.execution_results.modified_objects.contains(id))
320            .for_each(|id| {
321                unwrapped_then_deleted.push((
322                    *id,
323                    lampot_version,
324                    ObjectDigest::OBJECT_DIGEST_DELETED,
325                ));
326            });
327        modified_at_versions.extend(deleted_at_versions);
328
329        let inner = self.into_inner();
330        let effects = TransactionEffects::new_from_execution_v1(
331            status,
332            epoch,
333            gas_cost_summary,
334            modified_at_versions,
335            shared_object_refs,
336            *transaction_digest,
337            created,
338            mutated,
339            unwrapped,
340            deleted,
341            unwrapped_then_deleted,
342            wrapped,
343            updated_gas_object_info,
344            if inner.events.data.is_empty() {
345                None
346            } else {
347                Some(inner.events.digest())
348            },
349            transaction_dependencies.into_iter().collect(),
350        );
351        (inner, effects)
352    }
353
354    fn into_effects_v2(
355        self,
356        shared_object_refs: Vec<SharedInput>,
357        transaction_digest: &TransactionDigest,
358        transaction_dependencies: BTreeSet<TransactionDigest>,
359        gas_cost_summary: GasCostSummary,
360        status: ExecutionStatus,
361        gas_charger: &mut GasCharger,
362        epoch: EpochId,
363    ) -> (InnerTemporaryStore, TransactionEffects) {
364        // In the case of special transactions that don't require a gas object,
365        // we don't really care about the effects to gas, just use the input for it.
366        // Gas coins are guaranteed to be at least size 1 and if more than 1
367        // the first coin is where all the others are merged.
368        let gas_coin = gas_charger.gas_coin();
369
370        let object_changes = self.get_object_changes();
371
372        let lamport_version = self.lamport_timestamp;
373        let inner = self.into_inner();
374
375        let effects = TransactionEffects::new_from_execution_v2(
376            status,
377            epoch,
378            gas_cost_summary,
379            // TODO: Provide the list of read-only shared objects directly.
380            shared_object_refs,
381            BTreeSet::new(),
382            *transaction_digest,
383            lamport_version,
384            object_changes,
385            gas_coin,
386            if inner.events.data.is_empty() {
387                None
388            } else {
389                Some(inner.events.digest())
390            },
391            transaction_dependencies.into_iter().collect(),
392        );
393
394        (inner, effects)
395    }
396
397    /// An internal check of the invariants (will only fire in debug)
398    #[cfg(debug_assertions)]
399    fn check_invariants(&self) {
400        // Check not both deleted and written
401        debug_assert!(
402            {
403                self.execution_results
404                    .written_objects
405                    .keys()
406                    .all(|id| !self.execution_results.deleted_object_ids.contains(id))
407            },
408            "Object both written and deleted."
409        );
410
411        // Check all mutable inputs are modified
412        debug_assert!(
413            {
414                self.mutable_input_refs
415                    .keys()
416                    .all(|id| self.execution_results.modified_objects.contains(id))
417            },
418            "Mutable input not modified."
419        );
420
421        debug_assert!(
422            {
423                self.execution_results
424                    .written_objects
425                    .values()
426                    .all(|obj| obj.previous_transaction == self.tx_digest)
427            },
428            "Object previous transaction not properly set",
429        );
430    }
431
432    /// Mutate a mutable input object. This is used to mutate input objects outside of PT execution.
433    pub fn mutate_input_object(&mut self, object: Object) {
434        let id = object.id();
435        debug_assert!(self.input_objects.contains_key(&id));
436        debug_assert!(!object.is_immutable());
437        self.execution_results.modified_objects.insert(id);
438        self.execution_results.written_objects.insert(id, object);
439    }
440
441    /// Mutate a child object outside of PT. This should be used extremely rarely.
442    /// Currently it's only used by advance_epoch_safe_mode because it's all native
443    /// without PT. This should almost never be used otherwise.
444    pub fn mutate_child_object(&mut self, old_object: Object, new_object: Object) {
445        let id = new_object.id();
446        let old_ref = old_object.compute_object_reference();
447        debug_assert_eq!(old_ref.0, id);
448        self.loaded_runtime_objects.insert(
449            id,
450            DynamicallyLoadedObjectMetadata {
451                version: old_ref.1,
452                digest: old_ref.2,
453                owner: old_object.owner.clone(),
454                storage_rebate: old_object.storage_rebate,
455                previous_transaction: old_object.previous_transaction,
456            },
457        );
458        self.execution_results.modified_objects.insert(id);
459        self.execution_results
460            .written_objects
461            .insert(id, new_object);
462    }
463
464    /// Upgrade system package during epoch change. This requires special treatment
465    /// since the system package to be upgraded is not in the input objects.
466    /// We could probably fix above to make it less special.
467    pub fn upgrade_system_package(&mut self, package: Object) {
468        let id = package.id();
469        assert!(package.is_package() && is_system_package(id));
470        self.execution_results.modified_objects.insert(id);
471        self.execution_results.written_objects.insert(id, package);
472    }
473
474    /// Crate a new objcet. This is used to create objects outside of PT execution.
475    pub fn create_object(&mut self, object: Object) {
476        // Created mutable objects' versions are set to the store's lamport timestamp when it is
477        // committed to effects. Creating an object at a non-zero version risks violating the
478        // lamport timestamp invariant (that a transaction's lamport timestamp is strictly greater
479        // than all versions witnessed by the transaction).
480        debug_assert!(
481            object.is_immutable() || object.version() == SequenceNumber::MIN,
482            "Created mutable objects should not have a version set",
483        );
484        let id = object.id();
485        self.execution_results.created_object_ids.insert(id);
486        self.execution_results.written_objects.insert(id, object);
487    }
488
489    /// Delete a mutable input object. This is used to delete input objects outside of PT execution.
490    pub fn delete_input_object(&mut self, id: &ObjectID) {
491        // there should be no deletion after write
492        debug_assert!(!self.execution_results.written_objects.contains_key(id));
493        debug_assert!(self.input_objects.contains_key(id));
494        self.execution_results.modified_objects.insert(*id);
495        self.execution_results.deleted_object_ids.insert(*id);
496    }
497
498    pub fn drop_writes(&mut self) {
499        self.execution_results.drop_writes();
500    }
501
502    pub fn read_object(&self, id: &ObjectID) -> Option<&Object> {
503        // there should be no read after delete
504        debug_assert!(!self.execution_results.deleted_object_ids.contains(id));
505        self.execution_results
506            .written_objects
507            .get(id)
508            .or_else(|| self.input_objects.get(id))
509    }
510
511    pub fn save_loaded_runtime_objects(
512        &mut self,
513        loaded_runtime_objects: BTreeMap<ObjectID, DynamicallyLoadedObjectMetadata>,
514    ) {
515        #[cfg(debug_assertions)]
516        {
517            for (id, v1) in &loaded_runtime_objects {
518                if let Some(v2) = self.loaded_runtime_objects.get(id) {
519                    assert_eq!(v1, v2);
520                }
521            }
522            for (id, v1) in &self.loaded_runtime_objects {
523                if let Some(v2) = loaded_runtime_objects.get(id) {
524                    assert_eq!(v1, v2);
525                }
526            }
527        }
528        // Merge the two maps because we may be calling the execution engine more than once
529        // (e.g. in advance epoch transaction, where we may be publishing a new system package).
530        self.loaded_runtime_objects.extend(loaded_runtime_objects);
531    }
532
533    pub fn save_wrapped_object_containers(
534        &mut self,
535        wrapped_object_containers: BTreeMap<ObjectID, ObjectID>,
536    ) {
537        #[cfg(debug_assertions)]
538        {
539            for (id, container1) in &wrapped_object_containers {
540                if let Some(container2) = self.wrapped_object_containers.get(id) {
541                    assert_eq!(container1, container2);
542                }
543            }
544            for (id, container1) in &self.wrapped_object_containers {
545                if let Some(container2) = wrapped_object_containers.get(id) {
546                    assert_eq!(container1, container2);
547                }
548            }
549        }
550        // Merge the two maps because we may be calling the execution engine more than once
551        // (e.g. in advance epoch transaction, where we may be publishing a new system package).
552        self.wrapped_object_containers
553            .extend(wrapped_object_containers);
554    }
555
556    pub fn estimate_effects_size_upperbound(&self) -> usize {
557        if self.protocol_config.enable_effects_v2() {
558            TransactionEffects::estimate_effects_size_upperbound_v2(
559                self.execution_results.written_objects.len(),
560                self.execution_results.modified_objects.len(),
561                self.input_objects.len(),
562            )
563        } else {
564            let num_deletes = self.execution_results.deleted_object_ids.len()
565                + self
566                    .execution_results
567                    .modified_objects
568                    .iter()
569                    .filter(|id| {
570                        // Filter for wrapped objects.
571                        !self.execution_results.written_objects.contains_key(id)
572                            && !self.execution_results.deleted_object_ids.contains(id)
573                    })
574                    .count();
575            // In the worst case, the number of deps is equal to the number of input objects
576            TransactionEffects::estimate_effects_size_upperbound_v1(
577                self.execution_results.written_objects.len(),
578                self.mutable_input_refs.len(),
579                num_deletes,
580                self.input_objects.len(),
581            )
582        }
583    }
584
585    pub fn written_objects_size(&self) -> usize {
586        self.execution_results
587            .written_objects
588            .values()
589            .fold(0, |sum, obj| sum + obj.object_size_for_gas_metering())
590    }
591
592    /// If there are unmetered storage rebate (due to system transaction), we put them into
593    /// the storage rebate of 0x5 object.
594    /// TODO: This will not work for potential future new system transactions if 0x5 is not in the input.
595    /// We should fix this.
596    pub fn conserve_unmetered_storage_rebate(&mut self, unmetered_storage_rebate: u64) {
597        if unmetered_storage_rebate == 0 {
598            // If unmetered_storage_rebate is 0, we are most likely executing the genesis transaction.
599            // And in that case we cannot mutate the 0x5 object because it's newly created.
600            // And there is no storage rebate that needs distribution anyway.
601            return;
602        }
603        tracing::debug!(
604            "Amount of unmetered storage rebate from system tx: {:?}",
605            unmetered_storage_rebate
606        );
607        let mut system_state_wrapper = self
608            .read_object(&SUI_SYSTEM_STATE_OBJECT_ID)
609            .expect("0x5 object must be mutated in system tx with unmetered storage rebate")
610            .clone();
611        // In unmetered execution, storage_rebate field of mutated object must be 0.
612        // If not, we would be dropping SUI on the floor by overriding it.
613        assert_eq!(system_state_wrapper.storage_rebate, 0);
614        system_state_wrapper.storage_rebate = unmetered_storage_rebate;
615        self.mutate_input_object(system_state_wrapper);
616    }
617
618    /// Given an object ID, if it's not modified, returns None.
619    /// Otherwise returns its metadata, including version, digest, owner and storage rebate.
620    /// A modified object must be either a mutable input, or a loaded child object.
621    /// The only exception is when we upgrade system packages, in which case the upgraded
622    /// system packages are not part of input, but are modified.
623    fn get_object_modified_at(
624        &self,
625        object_id: &ObjectID,
626    ) -> Option<DynamicallyLoadedObjectMetadata> {
627        if self.execution_results.modified_objects.contains(object_id) {
628            Some(
629                self.mutable_input_refs
630                    .get(object_id)
631                    .map(
632                        |((version, digest), owner)| DynamicallyLoadedObjectMetadata {
633                            version: *version,
634                            digest: *digest,
635                            owner: owner.clone(),
636                            // It's guaranteed that a mutable input object is an input object.
637                            storage_rebate: self.input_objects[object_id].storage_rebate,
638                            previous_transaction: self.input_objects[object_id]
639                                .previous_transaction,
640                        },
641                    )
642                    .or_else(|| self.loaded_runtime_objects.get(object_id).cloned())
643                    .unwrap_or_else(|| {
644                        debug_assert!(is_system_package(*object_id));
645                        let package_obj =
646                            self.store.get_package_object(object_id).unwrap().unwrap();
647                        let obj = package_obj.object();
648                        DynamicallyLoadedObjectMetadata {
649                            version: obj.version(),
650                            digest: obj.digest(),
651                            owner: obj.owner.clone(),
652                            storage_rebate: obj.storage_rebate,
653                            previous_transaction: obj.previous_transaction,
654                        }
655                    }),
656            )
657        } else {
658            None
659        }
660    }
661}
662
663impl TemporaryStore<'_> {
664    // check that every object read is owned directly or indirectly by sender, sponsor,
665    // or a shared object input
666    pub fn check_ownership_invariants(
667        &self,
668        sender: &SuiAddress,
669        gas_charger: &mut GasCharger,
670        mutable_inputs: &HashSet<ObjectID>,
671        is_epoch_change: bool,
672    ) -> SuiResult<()> {
673        let gas_objs: HashSet<&ObjectID> = gas_charger.gas_coins().iter().map(|g| &g.0).collect();
674        // mark input objects as authenticated
675        let mut authenticated_for_mutation: HashSet<_> = self
676            .input_objects
677            .iter()
678            .filter_map(|(id, obj)| {
679                if gas_objs.contains(id) {
680                    // gas could be owned by either the sender (common case) or sponsor
681                    // (if this is a sponsored tx, which we do not know inside this function).
682                    // Either way, no object ownership chain should be rooted in a gas object
683                    // thus, consider object authenticated, but don't add it to authenticated_objs
684                    return None;
685                }
686                match &obj.owner {
687                    Owner::AddressOwner(a) => {
688                        assert!(sender == a, "Input object not owned by sender");
689                        Some(id)
690                    }
691                    Owner::Shared { .. } => Some(id),
692                    Owner::Immutable => {
693                        // object is authenticated, but it cannot own other objects,
694                        // so we should not add it to `authenticated_objs`
695                        // However, we would definitely want to add immutable objects
696                        // to the set of authenticated roots if we were doing runtime
697                        // checks inside the VM instead of after-the-fact in the temporary
698                        // store. Here, we choose not to add them because this will catch a
699                        // bug where we mutate or delete an object that belongs to an immutable
700                        // object (though it will show up somewhat opaquely as an authentication
701                        // failure), whereas adding the immutable object to the roots will prevent
702                        // us from catching this.
703                        None
704                    }
705                    Owner::ObjectOwner(_parent) => {
706                        unreachable!("Input objects must be address owned, shared, or immutable")
707                    }
708                    Owner::ConsensusAddressOwner { .. } => {
709                        unimplemented!(
710                            "ConsensusAddressOwner does not exist for this execution version"
711                        )
712                    }
713                }
714            })
715            .filter(|id| {
716                // remove any non-mutable inputs. This will remove deleted or readonly shared
717                // objects
718                mutable_inputs.contains(id)
719            })
720            .copied()
721            .collect();
722
723        // check all modified objects are authenticated (excluding gas objects)
724        let mut objects_to_authenticate = self
725            .execution_results
726            .modified_objects
727            .iter()
728            .filter(|id| !gas_objs.contains(id))
729            .copied()
730            .collect::<Vec<_>>();
731        // Map from an ObjectID to the ObjectID that covers it.
732        while let Some(to_authenticate) = objects_to_authenticate.pop() {
733            if authenticated_for_mutation.contains(&to_authenticate) {
734                // object has been authenticated
735                continue;
736            }
737            let wrapped_parent = self.wrapped_object_containers.get(&to_authenticate);
738            let parent = if let Some(container_id) = wrapped_parent {
739                // If the object is wrapped, then the container must be authenticated.
740                // For example, the ID is for a wrapped table or bag.
741                *container_id
742            } else {
743                let Some(old_obj) = self.store.get_object(&to_authenticate) else {
744                    panic!(
745                        "
746                        Failed to load object {to_authenticate:?}. \n\
747                        If it cannot be loaded, \
748                        we would expect it to be in the wrapped object map: {:?}",
749                        &self.wrapped_object_containers
750                    )
751                };
752                match &old_obj.owner {
753                    Owner::ObjectOwner(parent) => ObjectID::from(*parent),
754                    Owner::AddressOwner(parent) => {
755                        // For Receiving<_> objects, the address owner is actually an object.
756                        // If it was actually an address, we should have caught it as an input and
757                        // it would already have been in authenticated_for_mutation
758                        ObjectID::from(*parent)
759                    }
760                    owner @ Owner::Shared { .. } => panic!(
761                        "Unauthenticated root at {to_authenticate:?} with owner {owner:?}\n\
762                        Potentially covering objects in: {authenticated_for_mutation:#?}",
763                    ),
764                    Owner::Immutable => {
765                        assert!(
766                            is_epoch_change,
767                            "Immutable objects cannot be written, except for \
768                            Sui Framework/Move stdlib upgrades at epoch change boundaries"
769                        );
770                        // Note: this assumes that the only immutable objects an epoch change
771                        // tx can update are system packages,
772                        // but in principle we could allow others.
773                        assert!(
774                            is_system_package(to_authenticate),
775                            "Only system packages can be upgraded"
776                        );
777                        continue;
778                    }
779                    Owner::ConsensusAddressOwner { .. } => {
780                        unimplemented!(
781                            "ConsensusAddressOwner does not exist for this execution version"
782                        )
783                    }
784                }
785            };
786            // we now assume the object is authenticated and must check the parent
787            authenticated_for_mutation.insert(to_authenticate);
788            objects_to_authenticate.push(parent);
789        }
790        Ok(())
791    }
792}
793
794impl TemporaryStore<'_> {
795    /// Track storage gas for each mutable input object (including the gas coin)
796    /// and each created object. Compute storage refunds for each deleted object.
797    /// Will *not* charge anything, gas status keeps track of storage cost and rebate.
798    /// All objects will be updated with their new (current) storage rebate/cost.
799    /// `SuiGasStatus` `storage_rebate` and `storage_gas_units` track the transaction
800    /// overall storage rebate and cost.
801    pub(crate) fn collect_storage_and_rebate(&mut self, gas_charger: &mut GasCharger) {
802        // Use two loops because we cannot mut iterate written while calling get_object_modified_at.
803        let old_storage_rebates: Vec<_> = self
804            .execution_results
805            .written_objects
806            .keys()
807            .map(|object_id| {
808                self.get_object_modified_at(object_id)
809                    .map(|metadata| metadata.storage_rebate)
810                    .unwrap_or_default()
811            })
812            .collect();
813        for (object, old_storage_rebate) in self
814            .execution_results
815            .written_objects
816            .values_mut()
817            .zip(old_storage_rebates)
818        {
819            // new object size
820            let new_object_size = object.object_size_for_gas_metering();
821            // track changes and compute the new object `storage_rebate`
822            let new_storage_rebate = gas_charger.track_storage_mutation(
823                object.id(),
824                new_object_size,
825                old_storage_rebate,
826            );
827            object.storage_rebate = new_storage_rebate;
828        }
829
830        self.collect_rebate(gas_charger);
831    }
832
833    pub(crate) fn collect_rebate(&self, gas_charger: &mut GasCharger) {
834        for object_id in &self.execution_results.modified_objects {
835            if self
836                .execution_results
837                .written_objects
838                .contains_key(object_id)
839            {
840                continue;
841            }
842            // get and track the deleted object `storage_rebate`
843            let storage_rebate = self
844                .get_object_modified_at(object_id)
845                // Unwrap is safe because this loop iterates through all modified objects.
846                .unwrap()
847                .storage_rebate;
848            gas_charger.track_storage_mutation(*object_id, 0, storage_rebate);
849        }
850    }
851
852    pub fn check_execution_results_consistency(&self) -> Result<(), ExecutionError> {
853        assert_invariant!(
854            self.execution_results
855                .created_object_ids
856                .iter()
857                .all(|id| !self.execution_results.deleted_object_ids.contains(id)
858                    && !self.execution_results.modified_objects.contains(id)),
859            "Created object IDs cannot also be deleted or modified"
860        );
861        assert_invariant!(
862            self.execution_results.modified_objects.iter().all(|id| {
863                self.mutable_input_refs.contains_key(id)
864                    || self.loaded_runtime_objects.contains_key(id)
865                    || is_system_package(*id)
866            }),
867            "A modified object must be either a mutable input, a loaded child object, or a system package"
868        );
869        Ok(())
870    }
871}
872//==============================================================================
873// Charge gas current - end
874//==============================================================================
875
876impl TemporaryStore<'_> {
877    pub fn advance_epoch_safe_mode(
878        &mut self,
879        params: &AdvanceEpochParams,
880        protocol_config: &ProtocolConfig,
881    ) {
882        let wrapper = get_sui_system_state_wrapper(self.store.as_object_store())
883            .expect("System state wrapper object must exist");
884        let (old_object, new_object) =
885            wrapper.advance_epoch_safe_mode(params, self.store.as_object_store(), protocol_config);
886        self.mutate_child_object(old_object, new_object);
887    }
888}
889
890type ModifiedObjectInfo<'a> = (
891    ObjectID,
892    // old object metadata, including version, digest, owner, and storage rebate.
893    Option<DynamicallyLoadedObjectMetadata>,
894    Option<&'a Object>,
895);
896
897impl TemporaryStore<'_> {
898    fn get_input_sui(
899        &self,
900        id: &ObjectID,
901        expected_version: SequenceNumber,
902        layout_resolver: &mut impl LayoutResolver,
903    ) -> Result<u64, ExecutionError> {
904        if let Some(obj) = self.input_objects.get(id) {
905            // the assumption here is that if it is in the input objects must be the right one
906            if obj.version() != expected_version {
907                invariant_violation!(
908                    "Version mismatching when resolving input object to check conservation--\
909                     expected {}, got {}",
910                    expected_version,
911                    obj.version(),
912                );
913            }
914            obj.get_total_sui(layout_resolver).map_err(|e| {
915                make_invariant_violation!(
916                    "Failed looking up input SUI in SUI conservation checking for input with \
917                         type {:?}: {e:#?}",
918                    obj.struct_tag(),
919                )
920            })
921        } else {
922            // not in input objects, must be a dynamic field
923            let Some(obj) = self.store.get_object_by_key(id, expected_version) else {
924                invariant_violation!(
925                    "Failed looking up dynamic field {id} in SUI conservation checking"
926                );
927            };
928            obj.get_total_sui(layout_resolver).map_err(|e| {
929                make_invariant_violation!(
930                    "Failed looking up input SUI in SUI conservation checking for type \
931                         {:?}: {e:#?}",
932                    obj.struct_tag(),
933                )
934            })
935        }
936    }
937
938    /// Return the list of all modified objects, for each object, returns
939    /// - Object ID,
940    /// - Input: If the object existed prior to this transaction, include their version and storage_rebate,
941    /// - Output: If a new version of the object is written, include the new object.
942    fn get_modified_objects(&self) -> Vec<ModifiedObjectInfo<'_>> {
943        self.execution_results
944            .modified_objects
945            .iter()
946            .map(|id| {
947                let metadata = self.get_object_modified_at(id);
948                let output = self.execution_results.written_objects.get(id);
949                (*id, metadata, output)
950            })
951            .chain(
952                self.execution_results
953                    .written_objects
954                    .iter()
955                    .filter_map(|(id, object)| {
956                        if self.execution_results.modified_objects.contains(id) {
957                            None
958                        } else {
959                            Some((*id, None, Some(object)))
960                        }
961                    }),
962            )
963            .collect()
964    }
965
966    /// Check that this transaction neither creates nor destroys SUI. This should hold for all txes
967    /// except the epoch change tx, which mints staking rewards equal to the gas fees burned in the
968    /// previous epoch.  Specifically, this checks two key invariants about storage
969    /// fees and storage rebate:
970    ///
971    /// 1. all SUI in storage rebate fields of input objects should flow either to the transaction
972    ///    storage rebate, or the transaction non-refundable storage rebate
973    /// 2. all SUI charged for storage should flow into the storage rebate field of some output
974    ///    object
975    ///
976    /// This function is intended to be called *after* we have charged for
977    /// gas + applied the storage rebate to the gas object, but *before* we
978    /// have updated object versions.
979    pub fn check_sui_conserved(
980        &self,
981        simple_conservation_checks: bool,
982        gas_summary: &GasCostSummary,
983    ) -> Result<(), ExecutionError> {
984        if !simple_conservation_checks {
985            return Ok(());
986        }
987        // total amount of SUI in storage rebate of input objects
988        let mut total_input_rebate = 0;
989        // total amount of SUI in storage rebate of output objects
990        let mut total_output_rebate = 0;
991        for (_, input, output) in self.get_modified_objects() {
992            if let Some(input) = input {
993                total_input_rebate += input.storage_rebate;
994            }
995            if let Some(object) = output {
996                total_output_rebate += object.storage_rebate;
997            }
998        }
999
1000        if gas_summary.storage_cost == 0 {
1001            // this condition is usually true when the transaction went OOG and no
1002            // gas is left for storage charges.
1003            // The storage cost has to be there at least for the gas coin which
1004            // will not be deleted even when going to 0.
1005            // However if the storage cost is 0 and if there is any object touched
1006            // or deleted the value in input must be equal to the output plus rebate and
1007            // non refundable.
1008            // Rebate and non refundable will be positive when there are object deleted
1009            // (gas smashing being the primary and possibly only example).
1010            // A more typical condition is for all storage charges in summary to be 0 and
1011            // then input and output must be the same value
1012            if total_input_rebate
1013                != total_output_rebate
1014                    + gas_summary.storage_rebate
1015                    + gas_summary.non_refundable_storage_fee
1016            {
1017                return Err(ExecutionError::invariant_violation(format!(
1018                    "SUI conservation failed -- no storage charges in gas summary \
1019                        and total storage input rebate {} not equal  \
1020                        to total storage output rebate {}",
1021                    total_input_rebate, total_output_rebate,
1022                )));
1023            }
1024        } else {
1025            // all SUI in storage rebate fields of input objects should flow either to
1026            // the transaction storage rebate, or the non-refundable storage rebate pool
1027            if total_input_rebate
1028                != gas_summary.storage_rebate + gas_summary.non_refundable_storage_fee
1029            {
1030                return Err(ExecutionError::invariant_violation(format!(
1031                    "SUI conservation failed -- {} SUI in storage rebate field of input objects, \
1032                        {} SUI in tx storage rebate or tx non-refundable storage rebate",
1033                    total_input_rebate, gas_summary.non_refundable_storage_fee,
1034                )));
1035            }
1036
1037            // all SUI charged for storage should flow into the storage rebate field
1038            // of some output object
1039            if gas_summary.storage_cost != total_output_rebate {
1040                return Err(ExecutionError::invariant_violation(format!(
1041                    "SUI conservation failed -- {} SUI charged for storage, \
1042                        {} SUI in storage rebate field of output objects",
1043                    gas_summary.storage_cost, total_output_rebate
1044                )));
1045            }
1046        }
1047        Ok(())
1048    }
1049
1050    /// Check that this transaction neither creates nor destroys SUI.
1051    /// This more expensive check will check a third invariant on top of the 2 performed
1052    /// by `check_sui_conserved` above:
1053    ///
1054    /// * all SUI in input objects (including coins etc in the Move part of an object) should flow
1055    ///   either to an output object, or be burned as part of computation fees or non-refundable
1056    ///   storage rebate
1057    ///
1058    /// This function is intended to be called *after* we have charged for gas + applied the
1059    /// storage rebate to the gas object, but *before* we have updated object versions. The
1060    /// advance epoch transaction would mint `epoch_fees` amount of SUI, and burn `epoch_rebates`
1061    /// amount of SUI. We need these information for this check.
1062    pub fn check_sui_conserved_expensive(
1063        &self,
1064        gas_summary: &GasCostSummary,
1065        advance_epoch_gas_summary: Option<(u64, u64)>,
1066        layout_resolver: &mut impl LayoutResolver,
1067    ) -> Result<(), ExecutionError> {
1068        // total amount of SUI in input objects, including both coins and storage rebates
1069        let mut total_input_sui = 0;
1070        // total amount of SUI in output objects, including both coins and storage rebates
1071        let mut total_output_sui = 0;
1072        for (id, input, output) in self.get_modified_objects() {
1073            if let Some(input) = input {
1074                total_input_sui += self.get_input_sui(&id, input.version, layout_resolver)?;
1075            }
1076            if let Some(object) = output {
1077                total_output_sui += object.get_total_sui(layout_resolver).map_err(|e| {
1078                    make_invariant_violation!(
1079                        "Failed looking up output SUI in SUI conservation checking for \
1080                         mutated type {:?}: {e:#?}",
1081                        object.struct_tag(),
1082                    )
1083                })?;
1084            }
1085        }
1086        // note: storage_cost flows into the storage_rebate field of the output objects, which is
1087        // why it is not accounted for here.
1088        // similarly, all of the storage_rebate *except* the storage_fund_rebate_inflow
1089        // gets credited to the gas coin both computation costs and storage rebate inflow are
1090        total_output_sui += gas_summary.computation_cost + gas_summary.non_refundable_storage_fee;
1091        if let Some((epoch_fees, epoch_rebates)) = advance_epoch_gas_summary {
1092            total_input_sui += epoch_fees;
1093            total_output_sui += epoch_rebates;
1094        }
1095        if total_input_sui != total_output_sui {
1096            return Err(ExecutionError::invariant_violation(format!(
1097                "SUI conservation failed: input={}, output={}, \
1098                    this transaction either mints or burns SUI",
1099                total_input_sui, total_output_sui,
1100            )));
1101        }
1102        Ok(())
1103    }
1104}
1105
1106impl ChildObjectResolver for TemporaryStore<'_> {
1107    fn read_child_object(
1108        &self,
1109        parent: &ObjectID,
1110        child: &ObjectID,
1111        child_version_upper_bound: SequenceNumber,
1112    ) -> SuiResult<Option<Object>> {
1113        let obj_opt = self.execution_results.written_objects.get(child);
1114        if obj_opt.is_some() {
1115            Ok(obj_opt.cloned())
1116        } else {
1117            self.store
1118                .read_child_object(parent, child, child_version_upper_bound)
1119        }
1120    }
1121
1122    fn get_object_received_at_version(
1123        &self,
1124        owner: &ObjectID,
1125        receiving_object_id: &ObjectID,
1126        receive_object_at_version: SequenceNumber,
1127        epoch_id: EpochId,
1128    ) -> SuiResult<Option<Object>> {
1129        // You should never be able to try and receive an object after deleting it or writing it in the same
1130        // transaction since `Receiving` doesn't have copy.
1131        debug_assert!(!self
1132            .execution_results
1133            .written_objects
1134            .contains_key(receiving_object_id));
1135        debug_assert!(!self
1136            .execution_results
1137            .deleted_object_ids
1138            .contains(receiving_object_id));
1139        self.store.get_object_received_at_version(
1140            owner,
1141            receiving_object_id,
1142            receive_object_at_version,
1143            epoch_id,
1144        )
1145    }
1146}
1147
1148impl Storage for TemporaryStore<'_> {
1149    fn reset(&mut self) {
1150        self.drop_writes();
1151    }
1152
1153    fn read_object(&self, id: &ObjectID) -> Option<&Object> {
1154        TemporaryStore::read_object(self, id)
1155    }
1156
1157    /// Take execution results v2, and translate it back to be compatible with effects v1.
1158    fn record_execution_results(
1159        &mut self,
1160        results: ExecutionResults,
1161    ) -> Result<(), ExecutionError> {
1162        let ExecutionResults::V2(results) = results else {
1163            panic!("ExecutionResults::V2 expected in sui-execution v1 and above");
1164        };
1165        // It's important to merge instead of override results because it's
1166        // possible to execute PT more than once during tx execution.
1167        self.execution_results.merge_results(results);
1168        Ok(())
1169    }
1170
1171    fn save_loaded_runtime_objects(
1172        &mut self,
1173        loaded_runtime_objects: BTreeMap<ObjectID, DynamicallyLoadedObjectMetadata>,
1174    ) {
1175        TemporaryStore::save_loaded_runtime_objects(self, loaded_runtime_objects)
1176    }
1177
1178    fn save_wrapped_object_containers(
1179        &mut self,
1180        wrapped_object_containers: BTreeMap<ObjectID, ObjectID>,
1181    ) {
1182        TemporaryStore::save_wrapped_object_containers(self, wrapped_object_containers)
1183    }
1184
1185    fn check_coin_deny_list(
1186        &self,
1187        _receiving_funds_type_and_owners: BTreeMap<TypeTag, BTreeSet<SuiAddress>>,
1188    ) -> DenyListResult {
1189        unreachable!("Coin denylist v2 is not supported in sui-execution v2");
1190    }
1191
1192    fn record_generated_object_ids(&mut self, _generated_ids: BTreeSet<ObjectID>) {
1193        unreachable!(
1194            "Generated object IDs are not recorded in ExecutionResults in sui-execution v2"
1195        );
1196    }
1197}
1198
1199impl BackingPackageStore for TemporaryStore<'_> {
1200    fn get_package_object(&self, package_id: &ObjectID) -> SuiResult<Option<PackageObject>> {
1201        // We first check the objects in the temporary store because in non-production code path,
1202        // it is possible to read packages that are just written in the same transaction.
1203        // This can happen for example when we run the expensive conservation checks, where we may
1204        // look into the types of each written object in the output, and some of them need the
1205        // newly written packages for type checking.
1206        // In production path though, this should never happen.
1207        if let Some(obj) = self.execution_results.written_objects.get(package_id) {
1208            Ok(Some(PackageObject::new(obj.clone())))
1209        } else {
1210            self.store.get_package_object(package_id).inspect(|obj| {
1211                // Track object but leave unchanged
1212                if let Some(v) = obj {
1213                    if !self
1214                        .runtime_packages_loaded_from_db
1215                        .read()
1216                        .contains_key(package_id)
1217                    {
1218                        // TODO: Can this lock ever block execution?
1219                        // TODO: Another way to avoid the cost of maintaining this map is to not
1220                        // enable it in normal runs, and if a fork is detected, rerun it with a flag
1221                        // turned on and start populating this field.
1222                        self.runtime_packages_loaded_from_db
1223                            .write()
1224                            .insert(*package_id, v.clone());
1225                    }
1226                }
1227            })
1228        }
1229    }
1230}
1231
1232impl ParentSync for TemporaryStore<'_> {
1233    fn get_latest_parent_entry_ref_deprecated(&self, _object_id: ObjectID) -> Option<ObjectRef> {
1234        unreachable!("Never called in newer protocol versions")
1235    }
1236}