diff --git a/script/src/lib.rs b/script/src/lib.rs
index 3158a281b4..2016aeb3b1 100644
--- a/script/src/lib.rs
+++ b/script/src/lib.rs
@@ -10,9 +10,10 @@ mod verify_env;
pub use crate::error::{ScriptError, TransactionScriptError};
pub use crate::scheduler::{Scheduler, ROOT_VM_ID};
+pub use crate::syscalls::generator::generate_ckb_syscalls;
pub use crate::types::{
ChunkCommand, CoreMachine, DataPieceId, RunMode, ScriptGroup, ScriptGroupType, ScriptVersion,
TransactionState, TxData, VerifyResult, VmIsa, VmState, VmVersion,
};
-pub use crate::verify::{TransactionScriptsSyscallsGenerator, TransactionScriptsVerifier};
+pub use crate::verify::TransactionScriptsVerifier;
pub use crate::verify_env::TxVerifyEnv;
diff --git a/script/src/scheduler.rs b/script/src/scheduler.rs
index b83b1b5b9d..348b5caeaa 100644
--- a/script/src/scheduler.rs
+++ b/script/src/scheduler.rs
@@ -1,15 +1,13 @@
use crate::cost_model::transferred_byte_cycles;
use crate::syscalls::{
- EXEC_LOAD_ELF_V2_CYCLES_BASE, INVALID_FD, MAX_FDS_CREATED, MAX_VMS_SPAWNED, OTHER_END_CLOSED,
- SPAWN_EXTRA_CYCLES_BASE, SUCCESS, WAIT_FAILURE,
+ generator::generate_ckb_syscalls, EXEC_LOAD_ELF_V2_CYCLES_BASE, INVALID_FD, MAX_FDS_CREATED,
+ MAX_VMS_SPAWNED, OTHER_END_CLOSED, SPAWN_EXTRA_CYCLES_BASE, SUCCESS, WAIT_FAILURE,
};
-use crate::types::MachineContext;
-use crate::verify::TransactionScriptsSyscallsGenerator;
-use crate::ScriptVersion;
use crate::types::{
- CoreMachineType, DataLocation, DataPieceId, Fd, FdArgs, FullSuspendedState, Machine, Message,
- ReadState, RunMode, TxData, VmId, VmState, WriteState, FIRST_FD_SLOT, FIRST_VM_ID,
+ CoreMachineType, DataLocation, DataPieceId, DebugContext, Fd, FdArgs, FullSuspendedState,
+ Machine, Message, ReadState, RunMode, SgData, VmContext, VmId, VmState, WriteState,
+ FIRST_FD_SLOT, FIRST_VM_ID,
};
use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_types::core::Cycle;
@@ -25,7 +23,10 @@ use ckb_vm::{
Error, FlattenedArgsReader, Register,
};
use std::collections::{BTreeMap, HashMap};
-use std::sync::{Arc, Mutex};
+use std::sync::{
+ atomic::{AtomicU64, Ordering},
+ Arc, Mutex,
+};
/// Root process's id.
pub const ROOT_VM_ID: VmId = FIRST_VM_ID;
@@ -44,23 +45,45 @@ pub const MAX_FDS: u64 = 64;
/// of the core for IO operations.
pub struct Scheduler
where
- DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ DL: CellDataProvider,
{
- /// Context data for current running transaction & script.
- pub tx_data: TxData,
- /// In fact, Scheduler here has the potential to totally replace
- /// TransactionScriptsVerifier, nonetheless much of current syscall
- /// implementation is strictly tied to TransactionScriptsVerifier, we
- /// are using it here to save some extra code.
- pub script_version: ScriptVersion,
- /// Generate system calls.
- pub syscalls_generator: TransactionScriptsSyscallsGenerator,
-
- /// Total cycles.
- pub total_cycles: Cycle,
- /// Current iteration cycles. This value is periodically added to
- /// total_cycles and cleared
- pub current_iteration_cycles: Cycle,
+ /// Immutable context data for current running transaction & script.
+ pub sg_data: SgData,
+
+ /// Mutable context data used by current scheduler
+ pub debug_context: DebugContext,
+
+ /// Total cycles. When a scheduler executes, there are 3 variables
+ /// that might all contain charged cycles: +total_cycles+,
+ /// +iteration_cycles+ and +machine.cycles()+ from the current
+ /// executing virtual machine. At any given time, the sum of all 3
+ /// variables here, represent the total consumed cycles by the current
+ /// scheduler.
+ /// But there are also exceptions: at certain period of time, the cycles
+ /// stored in `machine.cycles()` are moved over to +iteration_cycles+,
+ /// the cycles stored in +iteration_cycles+ would also be moved over to
+ /// +total_cycles+:
+ ///
+ /// * The current running virtual machine would contain consumed
+ /// cycles in its own machine.cycles() structure.
+ /// * +iteration_cycles+ holds the current consumed cycles each time
+ /// we executed a virtual machine(also named an iteration). It will
+ /// always be zero before each iteration(i.e., before each VM starts
+ /// execution). When a virtual machine finishes execution, the cycles
+ /// stored in `machine.cycles()` will be moved over to +iteration_cycles+.
+ /// `machine.cycles()` will then be reset to zero.
+ /// * Processing messages in the message box would alao charge cycles
+ /// for operations, such as suspending/resuming VMs, transferring data
+ /// etc. Those cycles were added to +iteration_cycles+ directly. When all
+ /// postprocessing work is completed, the cycles consumed in
+ /// +iteration_cycles+ will then be moved to +total_cycles+.
+ /// +iteration_cycles+ will then be reset to zero.
+ ///
+ /// One can consider that +total_cycles+ contains the total cycles
+ /// consumed in current scheduler, when the scheduler is not busy executing.
+ pub total_cycles: Arc,
+ /// Iteration cycles, see +total_cycles+ on its usage
+ pub iteration_cycles: Cycle,
/// Next vm id used by spawn.
pub next_vm_id: VmId,
/// Next fd used by pipe.
@@ -72,7 +95,7 @@ where
/// Verify the VM's inherited fd list.
pub inherited_fd: BTreeMap>,
/// Instantiated vms.
- pub instantiated: BTreeMap, Machine)>,
+ pub instantiated: BTreeMap, Machine)>,
/// Suspended vms.
pub suspended: BTreeMap>,
/// Terminated vms.
@@ -88,18 +111,12 @@ where
DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
{
/// Create a new scheduler from empty state
- pub fn new(
- tx_data: TxData,
- script_version: ScriptVersion,
- syscalls_generator: TransactionScriptsSyscallsGenerator,
- ) -> Self {
- let message_box = Arc::clone(&syscalls_generator.message_box);
+ pub fn new(sg_data: SgData, debug_context: DebugContext) -> Self {
Self {
- tx_data,
- script_version,
- syscalls_generator,
- total_cycles: 0,
- current_iteration_cycles: 0,
+ sg_data,
+ debug_context,
+ total_cycles: Arc::new(AtomicU64::new(0)),
+ iteration_cycles: 0,
next_vm_id: FIRST_VM_ID,
next_fd_slot: FIRST_FD_SLOT,
states: BTreeMap::default(),
@@ -107,39 +124,39 @@ where
inherited_fd: BTreeMap::default(),
instantiated: BTreeMap::default(),
suspended: BTreeMap::default(),
- message_box,
+ message_box: Arc::new(Mutex::new(Vec::new())),
terminated_vms: BTreeMap::default(),
}
}
/// Return total cycles.
pub fn consumed_cycles(&self) -> Cycle {
- self.total_cycles
+ self.total_cycles.load(Ordering::Acquire)
}
/// Add cycles to total cycles.
- pub fn consumed_cycles_add(&mut self, cycles: Cycle) -> Result<(), Error> {
- self.total_cycles = self
+ pub fn consume_cycles(&mut self, cycles: Cycle) -> Result<(), Error> {
+ match self
.total_cycles
- .checked_add(cycles)
- .ok_or(Error::CyclesExceeded)?;
- Ok(())
+ .fetch_update(Ordering::AcqRel, Ordering::Acquire, |total_cycles| {
+ total_cycles.checked_add(cycles)
+ }) {
+ Ok(_) => Ok(()),
+ Err(_) => Err(Error::CyclesExceeded),
+ }
}
/// Resume a previously suspended scheduler state
pub fn resume(
- tx_data: TxData,
- script_version: ScriptVersion,
- syscalls_generator: TransactionScriptsSyscallsGenerator,
+ sg_data: SgData,
+ debug_context: DebugContext,
full: FullSuspendedState,
) -> Self {
- let message_box = Arc::clone(&syscalls_generator.message_box);
let mut scheduler = Self {
- tx_data,
- script_version,
- syscalls_generator,
- total_cycles: full.total_cycles,
- current_iteration_cycles: 0,
+ sg_data,
+ debug_context,
+ total_cycles: Arc::new(AtomicU64::new(full.total_cycles)),
+ iteration_cycles: 0,
next_vm_id: full.next_vm_id,
next_fd_slot: full.next_fd_slot,
states: full
@@ -155,12 +172,16 @@ where
.into_iter()
.map(|(id, _, snapshot)| (id, snapshot))
.collect(),
- message_box,
+ message_box: Arc::new(Mutex::new(Vec::new())),
terminated_vms: full.terminated_vms.into_iter().collect(),
};
scheduler
.ensure_vms_instantiated(&full.instantiated_ids)
.unwrap();
+ // NOTE: suspending/resuming a scheduler is part of CKB's implementation
+ // details. It is not part of execution consensue. We should not charge
+ // cycles for them.
+ scheduler.iteration_cycles = 0;
scheduler
}
@@ -180,7 +201,11 @@ where
vms.push((id, state, snapshot));
}
Ok(FullSuspendedState {
- total_cycles: self.total_cycles,
+ // NOTE: suspending a scheduler is actually part of CKB's
+ // internal execution logic, it does not belong to VM execution
+ // consensus. We are not charging cycles for suspending
+ // a VM in the process of suspending the whole scheduler.
+ total_cycles: self.total_cycles.load(Ordering::Acquire),
next_vm_id: self.next_vm_id,
next_fd_slot: self.next_fd_slot,
vms,
@@ -207,10 +232,11 @@ where
pub fn run(&mut self, mode: RunMode) -> Result<(i8, Cycle), Error> {
if self.states.is_empty() {
// Booting phase, we will need to initialize the first VM.
+ let program_id = self.sg_data.sg_info.program_data_piece_id.clone();
assert_eq!(
self.boot_vm(
&DataLocation {
- data_piece_id: DataPieceId::Program,
+ data_piece_id: program_id,
offset: 0,
length: u64::MAX,
},
@@ -227,26 +253,24 @@ where
};
while self.states[&ROOT_VM_ID] != VmState::Terminated {
- self.current_iteration_cycles = 0;
+ assert_eq!(self.iteration_cycles, 0);
let iterate_return = self.iterate(pause.clone(), limit_cycles);
- self.consumed_cycles_add(self.current_iteration_cycles)?;
+ self.consume_cycles(self.iteration_cycles)?;
limit_cycles = limit_cycles
- .checked_sub(self.current_iteration_cycles)
+ .checked_sub(self.iteration_cycles)
.ok_or(Error::CyclesExceeded)?;
+ // Clear iteration cycles intentionally after each run
+ self.iteration_cycles = 0;
iterate_return?;
}
// At this point, root VM cannot be suspended
let root_vm = &self.instantiated[&ROOT_VM_ID];
- Ok((root_vm.1.machine.exit_code(), self.total_cycles))
+ Ok((root_vm.1.machine.exit_code(), self.consumed_cycles()))
}
/// Returns the machine that needs to be executed in the current iterate.
- pub fn iterate_prepare_machine(
- &mut self,
- pause: Pause,
- limit_cycles: Cycle,
- ) -> Result<(u64, &mut Machine), Error> {
+ pub fn iterate_prepare_machine(&mut self) -> Result<(u64, &mut Machine), Error> {
// Process all pending VM reads & writes.
self.process_io()?;
// Find a runnable VM that has the largest ID.
@@ -260,11 +284,7 @@ where
let vm_id_to_run = vm_id_to_run.ok_or_else(|| {
Error::Unexpected("A deadlock situation has been reached!".to_string())
})?;
- let total_cycles = self.total_cycles;
- let (context, machine) = self.ensure_get_instantiated(&vm_id_to_run)?;
- context.set_base_cycles(total_cycles);
- machine.set_max_cycles(limit_cycles);
- machine.machine.set_pause(pause);
+ let (_context, machine) = self.ensure_get_instantiated(&vm_id_to_run)?;
Ok((vm_id_to_run, machine))
}
@@ -273,12 +293,7 @@ where
&mut self,
vm_id_to_run: u64,
result: Result,
- cycles: u64,
) -> Result<(), Error> {
- self.current_iteration_cycles = self
- .current_iteration_cycles
- .checked_add(cycles)
- .ok_or(Error::CyclesOverflow)?;
// Process message box, update VM states accordingly
self.process_message_box()?;
assert!(self.message_box.lock().expect("lock").is_empty());
@@ -336,11 +351,23 @@ where
// Here both pause signal and limit_cycles are provided so as to simplify
// branches.
fn iterate(&mut self, pause: Pause, limit_cycles: Cycle) -> Result<(), Error> {
- let (id, vm) = self.iterate_prepare_machine(pause, limit_cycles)?;
- let result = vm.run();
- let cycles = vm.machine.cycles();
- vm.machine.set_cycles(0);
- self.iterate_process_results(id, result, cycles)
+ // Execute the VM for real, consumed cycles in the virtual machine is
+ // moved over to +iteration_cycles+, then we reset virtual machine's own
+ // cycle count to zero.
+ let (id, result, cycles) = {
+ let (id, vm) = self.iterate_prepare_machine()?;
+ vm.set_max_cycles(limit_cycles);
+ vm.machine.set_pause(pause);
+ let result = vm.run();
+ let cycles = vm.machine.cycles();
+ vm.machine.set_cycles(0);
+ (id, result, cycles)
+ };
+ self.iteration_cycles = self
+ .iteration_cycles
+ .checked_add(cycles)
+ .ok_or(Error::CyclesExceeded)?;
+ self.iterate_process_results(id, result)
}
fn process_message_box(&mut self) -> Result<(), Error> {
@@ -358,7 +385,7 @@ where
let old_cycles = old_machine.machine.cycles();
let max_cycles = old_machine.machine.max_cycles();
let program = {
- let mut sc = old_context.snapshot2_context().lock().expect("lock");
+ let mut sc = old_context.snapshot2_context.lock().expect("lock");
sc.load_data(
&args.location.data_piece_id,
args.location.offset,
@@ -731,7 +758,7 @@ where
fn ensure_get_instantiated(
&mut self,
id: &VmId,
- ) -> Result<&mut (MachineContext, Machine), Error> {
+ ) -> Result<&mut (VmContext, Machine), Error> {
self.ensure_vms_instantiated(&[*id])?;
self.instantiated
.get_mut(id)
@@ -744,13 +771,13 @@ where
return Err(Error::Unexpected(format!("VM {:?} is not suspended!", id)));
}
let snapshot = &self.suspended[id];
- self.current_iteration_cycles = self
- .current_iteration_cycles
+ self.iteration_cycles = self
+ .iteration_cycles
.checked_add(SPAWN_EXTRA_CYCLES_BASE)
.ok_or(Error::CyclesExceeded)?;
let (context, mut machine) = self.create_dummy_vm(id)?;
{
- let mut sc = context.snapshot2_context().lock().expect("lock");
+ let mut sc = context.snapshot2_context.lock().expect("lock");
sc.resume(&mut machine.machine, snapshot)?;
}
self.instantiated.insert(*id, (context, machine));
@@ -766,8 +793,8 @@ where
id
)));
}
- self.current_iteration_cycles = self
- .current_iteration_cycles
+ self.iteration_cycles = self
+ .iteration_cycles
.checked_add(SPAWN_EXTRA_CYCLES_BASE)
.ok_or(Error::CyclesExceeded)?;
let (context, machine) = self
@@ -775,7 +802,7 @@ where
.get_mut(id)
.ok_or_else(|| Error::Unexpected("Unable to find VM Id".to_string()))?;
let snapshot = {
- let sc = context.snapshot2_context().lock().expect("lock");
+ let sc = context.snapshot2_context.lock().expect("lock");
sc.make_snapshot(&mut machine.machine)?
};
self.suspended.insert(*id, snapshot);
@@ -793,7 +820,7 @@ where
self.next_vm_id += 1;
let (context, mut machine) = self.create_dummy_vm(&id)?;
let (program, _) = {
- let mut sc = context.snapshot2_context().lock().expect("lock");
+ let mut sc = context.snapshot2_context.lock().expect("lock");
sc.load_data(&location.data_piece_id, location.offset, location.length)?
};
self.load_vm_program(&context, &mut machine, location, program, args)?;
@@ -807,6 +834,7 @@ where
.key();
self.suspend_vm(&id)?;
}
+
self.instantiated.insert(id, (context, machine));
self.states.insert(id, VmState::Runnable);
@@ -816,7 +844,7 @@ where
// Load the program into an empty vm.
fn load_vm_program(
&mut self,
- context: &MachineContext,
+ context: &VmContext,
machine: &mut Machine,
location: &DataLocation,
program: Bytes,
@@ -831,7 +859,7 @@ where
}
None => machine.load_program_with_metadata(&program, &metadata, vec![].into_iter())?,
};
- let mut sc = context.snapshot2_context().lock().expect("lock");
+ let mut sc = context.snapshot2_context.lock().expect("lock");
sc.mark_program(
&mut machine.machine,
&metadata,
@@ -845,35 +873,30 @@ where
}
// Create a new VM instance with syscalls attached
- fn create_dummy_vm(&self, id: &VmId) -> Result<(MachineContext, Machine), Error> {
+ fn create_dummy_vm(&self, id: &VmId) -> Result<(VmContext, Machine), Error> {
// The code here looks slightly weird, since I don't want to copy over all syscall
// impls here again. Ideally, this scheduler package should be merged with ckb-script,
// or simply replace ckb-script. That way, the quirks here will be eliminated.
- let version = self.script_version;
+ let version = &self.sg_data.sg_info.script_version;
let core_machine = CoreMachineType::new(
version.vm_isa(),
version.vm_version(),
// We will update max_cycles for each machine when it gets a chance to run
u64::MAX,
);
- let snapshot2_context = Arc::new(Mutex::new(Snapshot2Context::new(self.tx_data.clone())));
- let mut syscalls_generator = self.syscalls_generator.clone();
- syscalls_generator.vm_id = *id;
- let mut machine_context = MachineContext::new(self.tx_data.clone());
- machine_context.base_cycles = Arc::clone(&self.syscalls_generator.base_cycles);
- machine_context.snapshot2_context = Arc::clone(&snapshot2_context);
+ let vm_context = VmContext {
+ base_cycles: Arc::clone(&self.total_cycles),
+ message_box: Arc::clone(&self.message_box),
+ snapshot2_context: Arc::new(Mutex::new(Snapshot2Context::new(self.sg_data.clone()))),
+ };
let machine_builder = DefaultMachineBuilder::new(core_machine)
.instruction_cycle_func(Box::new(estimate_cycles));
- let machine_builder = syscalls_generator
- .generate_syscalls(
- version,
- &self.tx_data.script_group,
- Arc::clone(&snapshot2_context),
- )
- .into_iter()
- .fold(machine_builder, |builder, syscall| builder.syscall(syscall));
+ let machine_builder =
+ generate_ckb_syscalls(id, &self.sg_data, &vm_context, &self.debug_context)
+ .into_iter()
+ .fold(machine_builder, |builder, syscall| builder.syscall(syscall));
let default_machine = machine_builder.build();
- Ok((machine_context, Machine::new(default_machine)))
+ Ok((vm_context, Machine::new(default_machine)))
}
}
diff --git a/script/src/syscalls/close.rs b/script/src/syscalls/close.rs
index 1e359095be..536465b9da 100644
--- a/script/src/syscalls/close.rs
+++ b/script/src/syscalls/close.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{CLOSE, SPAWN_YIELD_CYCLES_BASE};
-use crate::types::{Fd, Message, VmId};
+use crate::types::{Fd, Message, VmContext, VmId};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct Close {
}
impl Close {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/current_cycles.rs b/script/src/syscalls/current_cycles.rs
index a791cef7a4..14e13da807 100644
--- a/script/src/syscalls/current_cycles.rs
+++ b/script/src/syscalls/current_cycles.rs
@@ -1,18 +1,27 @@
-use crate::syscalls::CURRENT_CYCLES;
+use crate::{syscalls::CURRENT_CYCLES, types::VmContext};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::{Arc, Mutex};
+use std::sync::{
+ atomic::{AtomicU64, Ordering},
+ Arc,
+};
#[derive(Debug, Default)]
pub struct CurrentCycles {
- base: Arc>,
+ base: Arc,
}
impl CurrentCycles {
- pub fn new(base: Arc>) -> Self {
- Self { base }
+ pub fn new(vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ base: Arc::clone(&vm_context.base_cycles),
+ }
}
}
@@ -27,8 +36,7 @@ impl Syscalls for CurrentCycles {
}
let cycles = self
.base
- .lock()
- .map_err(|e| VMError::Unexpected(e.to_string()))?
+ .load(Ordering::Acquire)
.checked_add(machine.cycles())
.ok_or(VMError::CyclesOverflow)?;
machine.set_register(A0, Mac::REG::from_u64(cycles));
diff --git a/script/src/syscalls/debugger.rs b/script/src/syscalls/debugger.rs
index 1b2158a7e7..75d2771638 100644
--- a/script/src/syscalls/debugger.rs
+++ b/script/src/syscalls/debugger.rs
@@ -1,19 +1,24 @@
-use crate::types::DebugPrinter;
+use crate::types::{
+ DebugContext, DebugPrinter, {SgData, SgInfo},
+};
use crate::{cost_model::transferred_byte_cycles, syscalls::DEBUG_PRINT_SYSCALL_NUMBER};
-use ckb_types::packed::Byte32;
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Memory, Register, SupportMachine, Syscalls,
};
+use std::sync::Arc;
pub struct Debugger {
- hash: Byte32,
+ sg_info: Arc,
printer: DebugPrinter,
}
impl Debugger {
- pub fn new(hash: Byte32, printer: DebugPrinter) -> Debugger {
- Debugger { hash, printer }
+ pub fn new(sg_data: &SgData, debug_context: &DebugContext) -> Debugger {
+ Debugger {
+ sg_info: Arc::clone(&sg_data.sg_info),
+ printer: Arc::clone(&debug_context.debug_printer),
+ }
}
}
@@ -46,7 +51,7 @@ impl Syscalls for Debugger {
machine.add_cycles_no_checking(transferred_byte_cycles(buffer.len() as u64))?;
let s = String::from_utf8(buffer)
.map_err(|e| VMError::External(format!("String from buffer {e:?}")))?;
- (self.printer)(&self.hash, s.as_str());
+ (self.printer)(&self.sg_info.script_hash, s.as_str());
Ok(true)
}
diff --git a/script/src/syscalls/exec.rs b/script/src/syscalls/exec.rs
index 1e934fddf8..ac1ce8cb70 100644
--- a/script/src/syscalls/exec.rs
+++ b/script/src/syscalls/exec.rs
@@ -3,9 +3,9 @@ use crate::syscalls::{
Place, Source, SourceEntry, EXEC, INDEX_OUT_OF_BOUND, MAX_ARGV_LENGTH, SLICE_OUT_OF_BOUND,
WRONG_FORMAT,
};
-use crate::types::Indices;
+use crate::types::SgData;
use ckb_traits::CellDataProvider;
-use ckb_types::core::cell::{CellMeta, ResolvedTransaction};
+use ckb_types::core::cell::CellMeta;
use ckb_types::core::error::ARGV_TOO_LONG_TEXT;
use ckb_types::packed::{Bytes as PackedBytes, BytesVec};
use ckb_vm::memory::load_c_string_byte_by_byte;
@@ -15,62 +15,49 @@ use ckb_vm::{
Error as VMError, Register, SupportMachine, Syscalls,
};
use ckb_vm::{DEFAULT_STACK_SIZE, RISCV_MAX_MEMORY};
-use std::sync::Arc;
#[derive(Debug)]
pub struct Exec {
- data_loader: DL,
- rtx: Arc,
- outputs: Arc>,
- group_inputs: Indices,
- group_outputs: Indices,
+ sg_data: SgData,
}
-impl Exec {
- pub fn new(
- data_loader: DL,
- rtx: Arc,
- outputs: Arc>,
- group_inputs: Indices,
- group_outputs: Indices,
- ) -> Exec {
+impl Exec {
+ pub fn new(sg_data: &SgData) -> Exec {
Exec {
- data_loader,
- rtx,
- outputs,
- group_inputs,
- group_outputs,
+ sg_data: sg_data.clone(),
}
}
#[inline]
fn resolved_inputs(&self) -> &Vec {
- &self.rtx.resolved_inputs
+ &self.sg_data.rtx.resolved_inputs
}
#[inline]
fn resolved_cell_deps(&self) -> &Vec {
- &self.rtx.resolved_cell_deps
+ &self.sg_data.rtx.resolved_cell_deps
}
#[inline]
fn witnesses(&self) -> BytesVec {
- self.rtx.transaction.witnesses()
+ self.sg_data.rtx.transaction.witnesses()
}
fn fetch_cell(&self, source: Source, index: usize) -> Result<&CellMeta, u8> {
let cell_opt = match source {
Source::Transaction(SourceEntry::Input) => self.resolved_inputs().get(index),
- Source::Transaction(SourceEntry::Output) => self.outputs.get(index),
+ Source::Transaction(SourceEntry::Output) => self.sg_data.outputs().get(index),
Source::Transaction(SourceEntry::CellDep) => self.resolved_cell_deps().get(index),
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.and_then(|actual_index| self.resolved_inputs().get(*actual_index)),
Source::Group(SourceEntry::Output) => self
- .group_outputs
+ .sg_data
+ .group_outputs()
.get(index)
- .and_then(|actual_index| self.outputs.get(*actual_index)),
+ .and_then(|actual_index| self.sg_data.outputs().get(*actual_index)),
Source::Transaction(SourceEntry::HeaderDep)
| Source::Group(SourceEntry::CellDep)
| Source::Group(SourceEntry::HeaderDep) => {
@@ -84,11 +71,13 @@ impl Exec {
fn fetch_witness(&self, source: Source, index: usize) -> Result {
let witness_opt = match source {
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.and_then(|actual_index| self.witnesses().get(*actual_index)),
Source::Group(SourceEntry::Output) => self
- .group_outputs
+ .sg_data
+ .group_outputs()
.get(index)
.and_then(|actual_index| self.witnesses().get(*actual_index)),
Source::Transaction(SourceEntry::Input) => self.witnesses().get(index),
@@ -102,7 +91,7 @@ impl Exec {
}
}
-impl Syscalls for Exec {
+impl Syscalls for Exec {
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
Ok(())
}
@@ -126,12 +115,15 @@ impl Syscalls for
return Ok(true);
}
let cell = cell.unwrap();
- self.data_loader.load_cell_data(cell).ok_or_else(|| {
- VMError::Unexpected(format!(
- "Unexpected load_cell_data failed {}",
- cell.out_point,
- ))
- })?
+ self.sg_data
+ .data_loader()
+ .load_cell_data(cell)
+ .ok_or_else(|| {
+ VMError::Unexpected(format!(
+ "Unexpected load_cell_data failed {}",
+ cell.out_point,
+ ))
+ })?
}
Place::Witness => {
let witness = self.fetch_witness(source, index as usize);
diff --git a/script/src/syscalls/exec_v2.rs b/script/src/syscalls/exec_v2.rs
index ab5b2facce..5f08ab39a9 100644
--- a/script/src/syscalls/exec_v2.rs
+++ b/script/src/syscalls/exec_v2.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{EXEC, INDEX_OUT_OF_BOUND};
-use crate::types::{DataLocation, DataPieceId, ExecV2Args, Message, VmId};
+use crate::types::{DataLocation, DataPieceId, ExecV2Args, Message, VmContext, VmId};
+use ckb_traits::CellDataProvider;
use ckb_vm::{
registers::{A0, A1, A2, A3, A4, A5, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -12,8 +13,11 @@ pub struct ExecV2 {
}
impl ExecV2 {
- pub fn new(id: VmId, message_box: Arc>>) -> ExecV2 {
- ExecV2 { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> ExecV2 {
+ ExecV2 {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/generator.rs b/script/src/syscalls/generator.rs
new file mode 100644
index 0000000000..98e8e7c1d5
--- /dev/null
+++ b/script/src/syscalls/generator.rs
@@ -0,0 +1,62 @@
+use crate::{
+ syscalls::{
+ Close, CurrentCycles, Debugger, Exec, ExecV2, InheritedFd, LoadBlockExtension, LoadCell,
+ LoadCellData, LoadHeader, LoadInput, LoadScript, LoadScriptHash, LoadTx, LoadWitness, Pipe,
+ ProcessID, Read, Spawn, VMVersion, Wait, Write,
+ },
+ types::{CoreMachine, DebugContext, ScriptVersion, SgData, VmContext, VmId},
+};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
+use ckb_vm::Syscalls;
+
+/// Generate RISC-V syscalls in CKB environment
+pub fn generate_ckb_syscalls(
+ vm_id: &VmId,
+ sg_data: &SgData,
+ vm_context: &VmContext,
+ debug_context: &DebugContext,
+) -> Vec)>>
+where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+{
+ let mut syscalls: Vec)>> = vec![
+ Box::new(LoadScriptHash::new(sg_data)),
+ Box::new(LoadTx::new(sg_data)),
+ Box::new(LoadCell::new(sg_data)),
+ Box::new(LoadInput::new(sg_data)),
+ Box::new(LoadHeader::new(sg_data)),
+ Box::new(LoadWitness::new(sg_data)),
+ Box::new(LoadScript::new(sg_data)),
+ Box::new(LoadCellData::new(vm_context)),
+ Box::new(Debugger::new(sg_data, debug_context)),
+ ];
+ let script_version = &sg_data.sg_info.script_version;
+ if script_version >= &ScriptVersion::V1 {
+ syscalls.append(&mut vec![
+ Box::new(VMVersion::new()),
+ Box::new(CurrentCycles::new(vm_context)),
+ ]);
+ }
+ if script_version == &ScriptVersion::V1 {
+ syscalls.push(Box::new(Exec::new(sg_data)));
+ }
+ if script_version >= &ScriptVersion::V2 {
+ syscalls.append(&mut vec![
+ Box::new(ExecV2::new(vm_id, vm_context)),
+ Box::new(LoadBlockExtension::new(sg_data)),
+ Box::new(Spawn::new(vm_id, vm_context)),
+ Box::new(ProcessID::new(vm_id)),
+ Box::new(Pipe::new(vm_id, vm_context)),
+ Box::new(Wait::new(vm_id, vm_context)),
+ Box::new(Write::new(vm_id, vm_context)),
+ Box::new(Read::new(vm_id, vm_context)),
+ Box::new(InheritedFd::new(vm_id, vm_context)),
+ Box::new(Close::new(vm_id, vm_context)),
+ ]);
+ }
+ #[cfg(test)]
+ syscalls.push(Box::new(crate::syscalls::Pause::new(
+ std::sync::Arc::clone(&debug_context.skip_pause),
+ )));
+ syscalls
+}
diff --git a/script/src/syscalls/inherited_fd.rs b/script/src/syscalls/inherited_fd.rs
index 4d041a0ba1..510b82969c 100644
--- a/script/src/syscalls/inherited_fd.rs
+++ b/script/src/syscalls/inherited_fd.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{INHERITED_FD, SPAWN_YIELD_CYCLES_BASE};
-use crate::types::{Fd, FdArgs, Message, VmId};
+use crate::types::{Fd, FdArgs, Message, VmContext, VmId};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A1, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct InheritedFd {
}
impl InheritedFd {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/load_block_extension.rs b/script/src/syscalls/load_block_extension.rs
index 061bb7c4c8..d38cf4b8cc 100644
--- a/script/src/syscalls/load_block_extension.rs
+++ b/script/src/syscalls/load_block_extension.rs
@@ -1,13 +1,12 @@
-use crate::types::Indices;
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{
utils::store_data, Source, SourceEntry, INDEX_OUT_OF_BOUND, ITEM_MISSING,
LOAD_BLOCK_EXTENSION, SUCCESS,
},
+ types::SgData,
};
use ckb_traits::ExtensionProvider;
-use ckb_types::core::cell::ResolvedTransaction;
use ckb_types::{
core::cell::CellMeta,
packed::{self, Byte32Vec},
@@ -16,41 +15,32 @@ use ckb_vm::{
registers::{A0, A3, A4, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::Arc;
#[derive(Debug)]
pub struct LoadBlockExtension {
- data_loader: DL,
- rtx: Arc,
- group_inputs: Indices,
+ sg_data: SgData,
}
-impl LoadBlockExtension {
- pub fn new(
- data_loader: DL,
- rtx: Arc,
- group_inputs: Indices,
- ) -> LoadBlockExtension {
+impl LoadBlockExtension {
+ pub fn new(sg_data: &SgData) -> LoadBlockExtension {
LoadBlockExtension {
- data_loader,
- rtx,
- group_inputs,
+ sg_data: sg_data.clone(),
}
}
#[inline]
fn header_deps(&self) -> Byte32Vec {
- self.rtx.transaction.header_deps()
+ self.sg_data.rtx.transaction.header_deps()
}
#[inline]
fn resolved_inputs(&self) -> &Vec {
- &self.rtx.resolved_inputs
+ &self.sg_data.rtx.resolved_inputs
}
#[inline]
fn resolved_cell_deps(&self) -> &Vec {
- &self.rtx.resolved_cell_deps
+ &self.sg_data.rtx.resolved_cell_deps
}
fn load_block_extension(&self, cell_meta: &CellMeta) -> Option {
@@ -64,7 +54,7 @@ impl LoadBlockExtension {
.into_iter()
.any(|hash| &hash == block_hash)
{
- self.data_loader.get_block_extension(block_hash)
+ self.sg_data.data_loader().get_block_extension(block_hash)
} else {
None
}
@@ -88,12 +78,14 @@ impl LoadBlockExtension {
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|block_hash| {
- self.data_loader
+ self.sg_data
+ .data_loader()
.get_block_extension(&block_hash)
.ok_or(ITEM_MISSING)
}),
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|actual_index| {
@@ -109,7 +101,7 @@ impl LoadBlockExtension {
}
}
-impl Syscalls
+impl Syscalls
for LoadBlockExtension
{
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
diff --git a/script/src/syscalls/load_cell.rs b/script/src/syscalls/load_cell.rs
index 0dee77937f..3c6212b1bd 100644
--- a/script/src/syscalls/load_cell.rs
+++ b/script/src/syscalls/load_cell.rs
@@ -1,18 +1,15 @@
-use crate::types::Indices;
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{
utils::store_data, CellField, Source, SourceEntry, INDEX_OUT_OF_BOUND, ITEM_MISSING,
LOAD_CELL_BY_FIELD_SYSCALL_NUMBER, LOAD_CELL_SYSCALL_NUMBER, SUCCESS,
},
+ types::{SgData, TxInfo},
};
use byteorder::{LittleEndian, WriteBytesExt};
use ckb_traits::CellDataProvider;
use ckb_types::{
- core::{
- cell::{CellMeta, ResolvedTransaction},
- Capacity,
- },
+ core::{cell::CellMeta, Capacity},
packed::CellOutput,
prelude::*,
};
@@ -20,41 +17,31 @@ use ckb_vm::{
registers::{A0, A3, A4, A5, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::Arc;
pub struct LoadCell {
- data_loader: DL,
- rtx: Arc,
- outputs: Arc>,
- group_inputs: Indices,
- group_outputs: Indices,
+ sg_data: SgData,
}
-impl LoadCell {
- pub fn new(
- data_loader: DL,
- rtx: Arc,
- outputs: Arc>,
- group_inputs: Indices,
- group_outputs: Indices,
- ) -> LoadCell {
+impl LoadCell {
+ pub fn new(sg_data: &SgData) -> LoadCell {
LoadCell {
- data_loader,
- rtx,
- outputs,
- group_inputs,
- group_outputs,
+ sg_data: sg_data.clone(),
}
}
+ #[inline]
+ fn tx_info(&self) -> &TxInfo {
+ &self.sg_data.tx_info
+ }
+
#[inline]
fn resolved_inputs(&self) -> &Vec {
- &self.rtx.resolved_inputs
+ &self.sg_data.rtx.resolved_inputs
}
#[inline]
fn resolved_cell_deps(&self) -> &Vec {
- &self.rtx.resolved_cell_deps
+ &self.sg_data.rtx.resolved_cell_deps
}
fn fetch_cell(&self, source: Source, index: usize) -> Result<&CellMeta, u8> {
@@ -63,7 +50,7 @@ impl LoadCell {
self.resolved_inputs().get(index).ok_or(INDEX_OUT_OF_BOUND)
}
Source::Transaction(SourceEntry::Output) => {
- self.outputs.get(index).ok_or(INDEX_OUT_OF_BOUND)
+ self.tx_info().outputs.get(index).ok_or(INDEX_OUT_OF_BOUND)
}
Source::Transaction(SourceEntry::CellDep) => self
.resolved_cell_deps()
@@ -71,7 +58,8 @@ impl LoadCell {
.ok_or(INDEX_OUT_OF_BOUND),
Source::Transaction(SourceEntry::HeaderDep) => Err(INDEX_OUT_OF_BOUND),
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|actual_index| {
@@ -80,10 +68,16 @@ impl LoadCell {
.ok_or(INDEX_OUT_OF_BOUND)
}),
Source::Group(SourceEntry::Output) => self
- .group_outputs
+ .sg_data
+ .group_outputs()
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
- .and_then(|actual_index| self.outputs.get(*actual_index).ok_or(INDEX_OUT_OF_BOUND)),
+ .and_then(|actual_index| {
+ self.tx_info()
+ .outputs
+ .get(*actual_index)
+ .ok_or(INDEX_OUT_OF_BOUND)
+ }),
Source::Group(SourceEntry::CellDep) => Err(INDEX_OUT_OF_BOUND),
Source::Group(SourceEntry::HeaderDep) => Err(INDEX_OUT_OF_BOUND),
}
@@ -115,7 +109,7 @@ impl LoadCell {
(SUCCESS, store_data(machine, &buffer)?)
}
CellField::DataHash => {
- if let Some(bytes) = self.data_loader.load_cell_data_hash(cell) {
+ if let Some(bytes) = self.tx_info().data_loader.load_cell_data_hash(cell) {
(SUCCESS, store_data(machine, &bytes.as_bytes())?)
} else {
(ITEM_MISSING, 0)
@@ -165,7 +159,9 @@ impl LoadCell {
}
}
-impl Syscalls for LoadCell {
+impl Syscalls
+ for LoadCell
+{
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
Ok(())
}
diff --git a/script/src/syscalls/load_cell_data.rs b/script/src/syscalls/load_cell_data.rs
index d9ee50be4b..3aa6b24645 100644
--- a/script/src/syscalls/load_cell_data.rs
+++ b/script/src/syscalls/load_cell_data.rs
@@ -1,4 +1,4 @@
-use crate::types::{DataPieceId, TxData};
+use crate::types::{DataPieceId, SgData, VmContext};
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{
@@ -20,17 +20,17 @@ pub struct LoadCellData
where
DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
{
- snapshot2_context: Arc>>>,
+ snapshot2_context: Arc>>>,
}
impl LoadCellData
where
DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
{
- pub fn new(
- snapshot2_context: Arc>>>,
- ) -> LoadCellData {
- LoadCellData { snapshot2_context }
+ pub fn new(vm_context: &VmContext) -> LoadCellData {
+ LoadCellData {
+ snapshot2_context: Arc::clone(&vm_context.snapshot2_context),
+ }
}
fn load_data(&self, machine: &mut Mac) -> Result<(), VMError> {
diff --git a/script/src/syscalls/load_header.rs b/script/src/syscalls/load_header.rs
index 41db861e7b..10629c844c 100644
--- a/script/src/syscalls/load_header.rs
+++ b/script/src/syscalls/load_header.rs
@@ -1,4 +1,3 @@
-use crate::types::Indices;
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{
@@ -6,9 +5,9 @@ use crate::{
HeaderField, Source, SourceEntry, INDEX_OUT_OF_BOUND, ITEM_MISSING,
LOAD_HEADER_BY_FIELD_SYSCALL_NUMBER, LOAD_HEADER_SYSCALL_NUMBER, SUCCESS,
},
+ types::SgData,
};
use ckb_traits::HeaderProvider;
-use ckb_types::core::cell::ResolvedTransaction;
use ckb_types::{
core::{cell::CellMeta, HeaderView},
packed::Byte32Vec,
@@ -18,45 +17,40 @@ use ckb_vm::{
registers::{A0, A3, A4, A5, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::Arc;
-
#[derive(Debug)]
pub struct LoadHeader {
- data_loader: DL,
- rtx: Arc,
- // This can only be used for liner search
- // header_deps: Byte32Vec,
- // resolved_inputs: &'a [CellMeta],
- // resolved_cell_deps: &'a [CellMeta],
- group_inputs: Indices,
+ sg_data: SgData,
}
-impl LoadHeader {
- pub fn new(
- data_loader: DL,
- rtx: Arc,
- group_inputs: Indices,
- ) -> LoadHeader {
+impl LoadHeader {
+ pub fn new(sg_data: &SgData) -> LoadHeader {
LoadHeader {
- data_loader,
- rtx,
- group_inputs,
+ sg_data: sg_data.clone(),
}
}
+ // This can only be used for liner search
+ // header_deps: Byte32Vec,
+ // resolved_inputs: &'a [CellMeta],
+ // resolved_cell_deps: &'a [CellMeta],
+ #[inline]
+ fn group_inputs(&self) -> &[usize] {
+ self.sg_data.group_inputs()
+ }
+
#[inline]
fn header_deps(&self) -> Byte32Vec {
- self.rtx.transaction.header_deps()
+ self.sg_data.rtx.transaction.header_deps()
}
#[inline]
fn resolved_inputs(&self) -> &Vec {
- &self.rtx.resolved_inputs
+ &self.sg_data.rtx.resolved_inputs
}
#[inline]
fn resolved_cell_deps(&self) -> &Vec {
- &self.rtx.resolved_cell_deps
+ &self.sg_data.rtx.resolved_cell_deps
}
fn load_header(&self, cell_meta: &CellMeta) -> Option {
@@ -70,7 +64,7 @@ impl LoadHeader {
.into_iter()
.any(|hash| &hash == block_hash)
{
- self.data_loader.get_header(block_hash)
+ self.sg_data.tx_info.data_loader.get_header(block_hash)
} else {
None
}
@@ -94,10 +88,14 @@ impl LoadHeader {
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|block_hash| {
- self.data_loader.get_header(&block_hash).ok_or(ITEM_MISSING)
+ self.sg_data
+ .tx_info
+ .data_loader
+ .get_header(&block_hash)
+ .ok_or(ITEM_MISSING)
}),
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .group_inputs()
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|actual_index| {
@@ -146,7 +144,9 @@ impl LoadHeader {
}
}
-impl Syscalls for LoadHeader {
+impl Syscalls
+ for LoadHeader
+{
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
Ok(())
}
diff --git a/script/src/syscalls/load_input.rs b/script/src/syscalls/load_input.rs
index 8f19c0d6c9..2d1f415f32 100644
--- a/script/src/syscalls/load_input.rs
+++ b/script/src/syscalls/load_input.rs
@@ -1,13 +1,12 @@
-use crate::types::Indices;
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{
utils::store_data, InputField, Source, SourceEntry, INDEX_OUT_OF_BOUND,
LOAD_INPUT_BY_FIELD_SYSCALL_NUMBER, LOAD_INPUT_SYSCALL_NUMBER, SUCCESS,
},
+ types::SgData,
};
use byteorder::{LittleEndian, WriteBytesExt};
-use ckb_types::core::cell::ResolvedTransaction;
use ckb_types::{
packed::{CellInput, CellInputVec},
prelude::*,
@@ -16,22 +15,22 @@ use ckb_vm::{
registers::{A0, A3, A4, A5, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::Arc;
#[derive(Debug)]
-pub struct LoadInput {
- rtx: Arc,
- group_inputs: Indices,
+pub struct LoadInput {
+ sg_data: SgData,
}
-impl LoadInput {
- pub fn new(rtx: Arc, group_inputs: Indices) -> LoadInput {
- LoadInput { rtx, group_inputs }
+impl LoadInput {
+ pub fn new(sg_data: &SgData) -> Self {
+ LoadInput {
+ sg_data: sg_data.clone(),
+ }
}
#[inline]
fn inputs(&self) -> CellInputVec {
- self.rtx.transaction.inputs()
+ self.sg_data.rtx.transaction.inputs()
}
fn fetch_input(&self, source: Source, index: usize) -> Result {
@@ -43,7 +42,8 @@ impl LoadInput {
Source::Transaction(SourceEntry::CellDep) => Err(INDEX_OUT_OF_BOUND),
Source::Transaction(SourceEntry::HeaderDep) => Err(INDEX_OUT_OF_BOUND),
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.ok_or(INDEX_OUT_OF_BOUND)
.and_then(|actual_index| {
@@ -87,7 +87,7 @@ impl LoadInput {
}
}
-impl Syscalls for LoadInput {
+impl Syscalls for LoadInput {
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
Ok(())
}
diff --git a/script/src/syscalls/load_script.rs b/script/src/syscalls/load_script.rs
index 6fa9f1f8d9..545e8e7492 100644
--- a/script/src/syscalls/load_script.rs
+++ b/script/src/syscalls/load_script.rs
@@ -1,21 +1,25 @@
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{utils::store_data, LOAD_SCRIPT_SYSCALL_NUMBER, SUCCESS},
+ types::{SgData, SgInfo},
};
-use ckb_types::{packed::Script, prelude::*};
+use ckb_types::prelude::*;
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
+use std::sync::Arc;
#[derive(Debug)]
pub struct LoadScript {
- script: Script,
+ sg_info: Arc,
}
impl LoadScript {
- pub fn new(script: Script) -> Self {
- Self { script }
+ pub fn new(sg_data: &SgData) -> Self {
+ Self {
+ sg_info: Arc::clone(&sg_data.sg_info),
+ }
}
}
@@ -29,7 +33,7 @@ impl Syscalls for LoadScript {
return Ok(false);
}
- let data = self.script.as_slice();
+ let data = self.sg_info.script_group.script.as_slice();
let wrote_size = store_data(machine, data)?;
machine.add_cycles_no_checking(transferred_byte_cycles(wrote_size))?;
diff --git a/script/src/syscalls/load_script_hash.rs b/script/src/syscalls/load_script_hash.rs
index d097cf0810..0fe93e2f30 100644
--- a/script/src/syscalls/load_script_hash.rs
+++ b/script/src/syscalls/load_script_hash.rs
@@ -1,21 +1,24 @@
use crate::{
cost_model::transferred_byte_cycles,
syscalls::{utils::store_data, LOAD_SCRIPT_HASH_SYSCALL_NUMBER, SUCCESS},
+ types::{SgData, SgInfo},
};
-use ckb_types::packed::Byte32;
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
+use std::sync::Arc;
#[derive(Debug)]
pub struct LoadScriptHash {
- hash: Byte32,
+ sg_info: Arc,
}
impl LoadScriptHash {
- pub fn new(hash: Byte32) -> LoadScriptHash {
- LoadScriptHash { hash }
+ pub fn new(sg_data: &SgData) -> LoadScriptHash {
+ LoadScriptHash {
+ sg_info: Arc::clone(&sg_data.sg_info),
+ }
}
}
@@ -29,7 +32,7 @@ impl Syscalls for LoadScriptHash {
return Ok(false);
}
- let data = self.hash.as_reader().raw_data();
+ let data = self.sg_info.script_hash.as_reader().raw_data();
let wrote_size = store_data(machine, data)?;
machine.add_cycles_no_checking(transferred_byte_cycles(wrote_size))?;
diff --git a/script/src/syscalls/load_tx.rs b/script/src/syscalls/load_tx.rs
index 5e933848fd..d0db1a3ce0 100644
--- a/script/src/syscalls/load_tx.rs
+++ b/script/src/syscalls/load_tx.rs
@@ -3,6 +3,7 @@ use crate::{
syscalls::{
utils::store_data, LOAD_TRANSACTION_SYSCALL_NUMBER, LOAD_TX_HASH_SYSCALL_NUMBER, SUCCESS,
},
+ types::SgData,
};
use ckb_types::{core::cell::ResolvedTransaction, prelude::*};
use ckb_vm::{
@@ -17,8 +18,10 @@ pub struct LoadTx {
}
impl LoadTx {
- pub fn new(rtx: Arc) -> LoadTx {
- LoadTx { rtx }
+ pub fn new(sg_data: &SgData) -> LoadTx {
+ LoadTx {
+ rtx: Arc::clone(&sg_data.rtx),
+ }
}
}
diff --git a/script/src/syscalls/load_witness.rs b/script/src/syscalls/load_witness.rs
index b5039be5c4..3ff4863199 100644
--- a/script/src/syscalls/load_witness.rs
+++ b/script/src/syscalls/load_witness.rs
@@ -4,51 +4,41 @@ use crate::{
utils::store_data, Source, SourceEntry, INDEX_OUT_OF_BOUND, LOAD_WITNESS_SYSCALL_NUMBER,
SUCCESS,
},
- types::Indices,
-};
-use ckb_types::{
- core::cell::ResolvedTransaction,
- packed::{Bytes, BytesVec},
+ types::SgData,
};
+use ckb_types::packed::{Bytes, BytesVec};
use ckb_vm::{
registers::{A0, A3, A4, A7},
Error as VMError, Register, SupportMachine, Syscalls,
};
-use std::sync::Arc;
#[derive(Debug)]
-pub struct LoadWitness {
- rtx: Arc,
- group_inputs: Indices,
- group_outputs: Indices,
+pub struct LoadWitness {
+ sg_data: SgData,
}
-impl LoadWitness {
- pub fn new(
- rtx: Arc,
- group_inputs: Indices,
- group_outputs: Indices,
- ) -> LoadWitness {
+impl LoadWitness {
+ pub fn new(sg_data: &SgData) -> Self {
LoadWitness {
- rtx,
- group_inputs,
- group_outputs,
+ sg_data: sg_data.clone(),
}
}
#[inline]
fn witnesses(&self) -> BytesVec {
- self.rtx.transaction.witnesses()
+ self.sg_data.rtx.transaction.witnesses()
}
fn fetch_witness(&self, source: Source, index: usize) -> Option {
match source {
Source::Group(SourceEntry::Input) => self
- .group_inputs
+ .sg_data
+ .group_inputs()
.get(index)
.and_then(|actual_index| self.witnesses().get(*actual_index)),
Source::Group(SourceEntry::Output) => self
- .group_outputs
+ .sg_data
+ .group_outputs()
.get(index)
.and_then(|actual_index| self.witnesses().get(*actual_index)),
Source::Transaction(SourceEntry::Input) => self.witnesses().get(index),
@@ -58,7 +48,7 @@ impl LoadWitness {
}
}
-impl Syscalls for LoadWitness {
+impl Syscalls for LoadWitness {
fn initialize(&mut self, _machine: &mut Mac) -> Result<(), VMError> {
Ok(())
}
diff --git a/script/src/syscalls/mod.rs b/script/src/syscalls/mod.rs
index 24a4ddcef3..a0ed55c168 100644
--- a/script/src/syscalls/mod.rs
+++ b/script/src/syscalls/mod.rs
@@ -28,6 +28,8 @@ mod pause;
#[cfg(test)]
mod tests;
+pub mod generator;
+
pub use self::close::Close;
pub use self::current_cycles::CurrentCycles;
pub use self::debugger::Debugger;
diff --git a/script/src/syscalls/pipe.rs b/script/src/syscalls/pipe.rs
index 3bb61ba22d..db7004b548 100644
--- a/script/src/syscalls/pipe.rs
+++ b/script/src/syscalls/pipe.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{PIPE, SPAWN_YIELD_CYCLES_BASE};
-use crate::types::{Message, PipeArgs, VmId};
+use crate::types::{Message, PipeArgs, VmContext, VmId};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct Pipe {
}
impl Pipe {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/process_id.rs b/script/src/syscalls/process_id.rs
index 55114ae18a..37b4df21e3 100644
--- a/script/src/syscalls/process_id.rs
+++ b/script/src/syscalls/process_id.rs
@@ -1,4 +1,5 @@
use crate::syscalls::PROCESS_ID;
+use crate::types::VmId;
use ckb_vm::{
registers::{A0, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -10,8 +11,8 @@ pub struct ProcessID {
}
impl ProcessID {
- pub fn new(id: u64) -> Self {
- Self { id }
+ pub fn new(vm_id: &VmId) -> Self {
+ Self { id: *vm_id }
}
}
diff --git a/script/src/syscalls/read.rs b/script/src/syscalls/read.rs
index 63976ef096..7c96c7125b 100644
--- a/script/src/syscalls/read.rs
+++ b/script/src/syscalls/read.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{INVALID_FD, READ, SPAWN_YIELD_CYCLES_BASE};
-use crate::types::{Fd, FdArgs, Message, VmId};
+use crate::types::{Fd, FdArgs, Message, VmContext, VmId};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A1, A2, A7},
Error as VMError, Memory, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct Read {
}
impl Read {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/spawn.rs b/script/src/syscalls/spawn.rs
index f4c7e82b87..a90ed3082a 100644
--- a/script/src/syscalls/spawn.rs
+++ b/script/src/syscalls/spawn.rs
@@ -2,7 +2,7 @@ use crate::syscalls::{
Source, INDEX_OUT_OF_BOUND, SLICE_OUT_OF_BOUND, SOURCE_ENTRY_MASK, SOURCE_GROUP_FLAG, SPAWN,
SPAWN_EXTRA_CYCLES_BASE, SPAWN_YIELD_CYCLES_BASE,
};
-use crate::types::{DataLocation, DataPieceId, Fd, Message, SpawnArgs, TxData, VmId};
+use crate::types::{DataLocation, DataPieceId, Fd, Message, SgData, SpawnArgs, VmContext, VmId};
use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
machine::SupportMachine,
@@ -20,22 +20,18 @@ where
{
id: VmId,
message_box: Arc>>,
- snapshot2_context: Arc>>>,
+ snapshot2_context: Arc>>>,
}
impl Spawn
where
DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
{
- pub fn new(
- id: VmId,
- message_box: Arc>>,
- snapshot2_context: Arc>>>,
- ) -> Self {
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self {
Self {
- id,
- message_box,
- snapshot2_context,
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ snapshot2_context: Arc::clone(&vm_context.snapshot2_context),
}
}
}
diff --git a/script/src/syscalls/tests/utils.rs b/script/src/syscalls/tests/utils.rs
index de9ca6f542..4f2da451e9 100644
--- a/script/src/syscalls/tests/utils.rs
+++ b/script/src/syscalls/tests/utils.rs
@@ -1,11 +1,22 @@
+use crate::{
+ types::{
+ DataPieceId, ScriptGroup, ScriptGroupType, ScriptVersion, SgData, SgInfo, TxData, TxInfo,
+ },
+ verify_env::TxVerifyEnv,
+};
+use ckb_chain_spec::consensus::ConsensusBuilder;
use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_types::{
bytes::Bytes,
- core::{cell::CellMeta, Capacity, HeaderView},
- packed::{self, Byte32, CellOutput, OutPoint},
+ core::{
+ cell::{CellMeta, ResolvedTransaction},
+ Capacity, HeaderBuilder, HeaderView,
+ },
+ packed::{self, Byte32, CellOutput, OutPoint, Script},
prelude::*,
};
-use std::collections::HashMap;
+use std::collections::{BTreeMap, HashMap};
+use std::sync::Arc;
#[derive(Default, Clone)]
pub(crate) struct MockDataLoader {
@@ -52,3 +63,79 @@ pub(crate) fn build_cell_meta(capacity_bytes: usize, data: Bytes) -> CellMeta {
mem_cell_data_hash: Some(data_hash),
}
}
+
+fn build_tx_data_with_loader(
+ rtx: Arc,
+ data_loader: MockDataLoader,
+) -> TxData {
+ let consensus = ConsensusBuilder::default().build();
+ let tx_env = TxVerifyEnv::new_commit(&HeaderBuilder::default().build());
+
+ TxData {
+ rtx,
+ info: Arc::new(TxInfo {
+ data_loader,
+ consensus: Arc::new(consensus),
+ tx_env: Arc::new(tx_env),
+ binaries_by_data_hash: HashMap::default(),
+ binaries_by_type_hash: HashMap::default(),
+ lock_groups: BTreeMap::default(),
+ type_groups: BTreeMap::default(),
+ outputs: Vec::new(),
+ }),
+ }
+}
+
+pub(crate) fn build_sg_data(
+ rtx: Arc,
+ input_indices: Vec,
+ output_indices: Vec,
+) -> SgData {
+ build_sg_data_with_loader(rtx, new_mock_data_loader(), input_indices, output_indices)
+}
+
+pub(crate) fn build_sg_data_with_loader(
+ rtx: Arc,
+ data_loader: MockDataLoader,
+ input_indices: Vec,
+ output_indices: Vec,
+) -> SgData {
+ let tx_data = build_tx_data_with_loader(rtx, data_loader);
+ let script_group = ScriptGroup {
+ script: Script::default(),
+ group_type: ScriptGroupType::Lock,
+ input_indices,
+ output_indices,
+ };
+ let script_hash = script_group.script.calc_script_hash();
+ SgData {
+ rtx: tx_data.rtx,
+ tx_info: tx_data.info,
+ sg_info: Arc::new(SgInfo {
+ script_version: ScriptVersion::latest(),
+ script_group,
+ script_hash,
+ program_data_piece_id: DataPieceId::CellDep(0),
+ }),
+ }
+}
+
+pub(crate) fn update_tx_info)>(
+ mut sg_data: SgData,
+ f: F,
+) -> SgData {
+ let mut tx_info = sg_data.tx_info.as_ref().clone();
+ f(&mut tx_info);
+ sg_data.tx_info = Arc::new(tx_info);
+ sg_data
+}
+
+pub(crate) fn update_sg_info(
+ mut sg_data: SgData,
+ f: F,
+) -> SgData {
+ let mut sg_info = sg_data.sg_info.as_ref().clone();
+ f(&mut sg_info);
+ sg_data.sg_info = Arc::new(sg_info);
+ sg_data
+}
diff --git a/script/src/syscalls/tests/vm_latest/syscalls_1.rs b/script/src/syscalls/tests/vm_latest/syscalls_1.rs
index f7ca26ecac..98f7843dcc 100644
--- a/script/src/syscalls/tests/vm_latest/syscalls_1.rs
+++ b/script/src/syscalls/tests/vm_latest/syscalls_1.rs
@@ -1,3 +1,4 @@
+use crate::types::VmContext;
use byteorder::{ByteOrder, LittleEndian, WriteBytesExt};
use ckb_hash::blake2b_256;
use ckb_types::{
@@ -14,7 +15,6 @@ use ckb_types::{
use ckb_vm::{
memory::{FLAG_DIRTY, FLAG_EXECUTABLE, FLAG_FREEZED, FLAG_WRITABLE},
registers::{A0, A1, A2, A3, A4, A5, A7},
- snapshot2::Snapshot2Context,
CoreMachine, Error as VMError, Memory, Syscalls, RISCV_PAGESIZE,
};
use proptest::{collection::size_range, prelude::*};
@@ -23,8 +23,6 @@ use std::sync::{Arc, Mutex};
use super::SCRIPT_VERSION;
use crate::syscalls::{tests::utils::*, *};
-use crate::types::TxData;
-use crate::types::{ScriptGroup, ScriptGroupType};
fn _test_load_cell_not_exist(data: &[u8]) -> Result<(), TestCaseError> {
let mut machine = SCRIPT_VERSION.init_core_machine_without_limit();
@@ -47,10 +45,6 @@ fn _test_load_cell_not_exist(data: &[u8]) -> Result<(), TestCaseError> {
let output = build_cell_meta(100, output_cell_data);
let input_cell_data: Bytes = data.iter().rev().cloned().collect();
let input_cell = build_cell_meta(100, input_cell_data);
- let outputs = Arc::new(vec![output]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -59,7 +53,10 @@ fn _test_load_cell_not_exist(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(load_cell.ecall(&mut machine).is_ok());
prop_assert_eq!(machine.registers()[A0], u64::from(INDEX_OUT_OF_BOUND));
@@ -89,10 +86,6 @@ fn _test_load_cell_all(data: &[u8]) -> Result<(), TestCaseError> {
let output = build_cell_meta(100, output_cell_data);
let input_cell_data: Bytes = data.iter().rev().cloned().collect();
let input_cell = build_cell_meta(100, input_cell_data);
- let outputs = Arc::new(vec![output.clone()]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -101,7 +94,10 @@ fn _test_load_cell_all(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
let input_correct_data = input_cell.cell_output.as_slice();
let output_correct_data = output.cell_output.as_slice();
@@ -179,10 +175,6 @@ fn _test_load_cell_from_group(data: &[u8], source: SourceEntry) -> Result<(), Te
let output = build_cell_meta(100, output_cell_data);
let input_cell_data: Bytes = data.iter().rev().cloned().collect();
let input_cell = build_cell_meta(100, input_cell_data);
- let outputs = Arc::new(vec![output.clone()]);
- let group_inputs = Arc::new(vec![0]);
- let group_outputs = Arc::new(vec![0]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -191,7 +183,10 @@ fn _test_load_cell_from_group(data: &[u8], source: SourceEntry) -> Result<(), Te
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
let input_correct_data = input_cell.cell_output.as_slice();
let output_correct_data = output.cell_output.as_slice();
@@ -267,11 +262,6 @@ fn _test_load_cell_out_of_bound(index: u64, source: u64) -> Result<(), TestCaseE
let input_cell = build_cell_meta(100, data);
- let outputs = Arc::new(vec![output]);
- let group_inputs = Arc::new(vec![0]);
- let group_outputs = Arc::new(vec![0]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -279,7 +269,11 @@ fn _test_load_cell_out_of_bound(index: u64, source: u64) -> Result<(), TestCaseE
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
+
prop_assert!(load_cell.ecall(&mut machine).is_ok());
prop_assert_eq!(machine.registers()[A0], u64::from(INDEX_OUT_OF_BOUND));
Ok(())
@@ -325,11 +319,6 @@ fn _test_load_cell_length(data: &[u8]) -> Result<(), TestCaseError> {
let input_cell_data: Bytes = data.iter().rev().cloned().collect();
let input_cell = build_cell_meta(100, input_cell_data);
- let outputs = Arc::new(vec![output]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -337,7 +326,10 @@ fn _test_load_cell_length(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
let input_correct_data = input_cell.cell_output.as_slice();
@@ -378,11 +370,6 @@ fn _test_load_cell_partial(data: &[u8], offset: u64) -> Result<(), TestCaseError
let input_cell_data: Bytes = data.iter().rev().cloned().collect();
let input_cell = build_cell_meta(100, input_cell_data);
- let outputs = Arc::new(vec![output]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -390,7 +377,10 @@ fn _test_load_cell_partial(data: &[u8], offset: u64) -> Result<(), TestCaseError
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ let sg_data = update_tx_info(sg_data, |tx_info| tx_info.outputs = vec![output.clone()]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
let input_correct_data = input_cell.cell_output.as_slice();
@@ -444,11 +434,6 @@ fn _test_load_cell_capacity(capacity: Capacity) -> Result<(), TestCaseError> {
mem_cell_data_hash: Some(data_hash),
};
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -456,7 +441,9 @@ fn _test_load_cell_capacity(capacity: Capacity) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine.memory_mut().store64(&size_addr, &16).is_ok());
@@ -505,11 +492,6 @@ fn _test_load_cell_occupied_capacity(data: &[u8]) -> Result<(), TestCaseError> {
mem_cell_data_hash: Some(data_hash),
};
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -517,7 +499,9 @@ fn _test_load_cell_occupied_capacity(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine.memory_mut().store64(&size_addr, &16).is_ok());
@@ -566,10 +550,6 @@ fn test_load_missing_data_hash() {
mem_cell_data: None,
mem_cell_data_hash: None,
};
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -578,7 +558,9 @@ fn test_load_missing_data_hash() {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
assert!(machine.memory_mut().store64(&size_addr, &100).is_ok());
@@ -612,10 +594,6 @@ fn _test_load_missing_contract(field: CellField) {
machine.set_register(A7, LOAD_CELL_BY_FIELD_SYSCALL_NUMBER); // syscall number
let output_cell = build_cell_meta(100, Bytes::new());
- let outputs = Arc::new(vec![output_cell]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -624,7 +602,12 @@ fn _test_load_missing_contract(field: CellField) {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ let sg_data = update_tx_info(sg_data, |tx_info| {
+ tx_info.outputs = vec![output_cell.clone()]
+ });
+
+ let mut load_cell = LoadCell::new(&sg_data);
assert!(machine.memory_mut().store64(&size_addr, &100).is_ok());
@@ -683,7 +666,6 @@ fn _test_load_header(
headers,
..Default::default()
};
- let group_inputs = Arc::new(vec![0]);
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
@@ -694,7 +676,9 @@ fn _test_load_header(
resolved_dep_groups: vec![],
});
- let mut load_header = LoadHeader::new(data_loader, rtx, group_inputs);
+ let sg_data = build_sg_data_with_loader(rtx, data_loader, vec![0], vec![]);
+
+ let mut load_header = LoadHeader::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -800,7 +784,6 @@ fn _test_load_header_by_field(data: &[u8], field: HeaderField) -> Result<(), Tes
headers,
..Default::default()
};
- let group_inputs = Arc::new(vec![]);
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
.header_dep(header.hash())
@@ -810,7 +793,9 @@ fn _test_load_header_by_field(data: &[u8], field: HeaderField) -> Result<(), Tes
resolved_dep_groups: vec![],
});
- let mut load_header = LoadHeader::new(data_loader, rtx, group_inputs);
+ let sg_data = build_sg_data_with_loader(rtx, data_loader, vec![], vec![]);
+
+ let mut load_header = LoadHeader::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -860,7 +845,9 @@ fn _test_load_tx_hash(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_tx = LoadTx::new(rtx);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_tx = LoadTx::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -911,7 +898,9 @@ fn _test_load_tx(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_tx = LoadTx::new(rtx);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_tx = LoadTx::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -955,7 +944,22 @@ fn _test_load_current_script_hash(data: &[u8]) -> Result<(), TestCaseError> {
.build();
let hash = script.calc_script_hash();
let data = hash.raw_data();
- let mut load_script_hash = LoadScriptHash::new(hash);
+
+ let rtx = Arc::new(ResolvedTransaction {
+ transaction: TransactionBuilder::default().build(),
+ resolved_cell_deps: vec![],
+ resolved_inputs: vec![],
+ resolved_dep_groups: vec![],
+ });
+
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ // Swap the internal script in VmData
+ let sg_data = update_sg_info(sg_data, |sg_info| {
+ sg_info.script_hash = script.calc_script_hash();
+ sg_info.script_group.script = script.clone();
+ });
+
+ let mut load_script_hash = LoadScriptHash::new(&sg_data);
prop_assert!(machine.memory_mut().store64(&size_addr, &64).is_ok());
@@ -1020,11 +1024,6 @@ fn _test_load_input_lock_script_hash(data: &[u8]) -> Result<(), TestCaseError> {
.build();
input_cell.cell_output = output_with_lock;
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -1032,7 +1031,9 @@ fn _test_load_input_lock_script_hash(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine.memory_mut().store64(&size_addr, &64).is_ok());
@@ -1085,11 +1086,6 @@ fn _test_load_input_lock_script(data: &[u8]) -> Result<(), TestCaseError> {
.build();
input_cell.cell_output = output_with_lock;
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -1097,7 +1093,9 @@ fn _test_load_input_lock_script(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1153,11 +1151,6 @@ fn _test_load_input_type_script(data: &[u8]) -> Result<(), TestCaseError> {
.build();
input_cell.cell_output = output_with_type;
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![],
@@ -1165,7 +1158,9 @@ fn _test_load_input_type_script(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1222,10 +1217,6 @@ fn _test_load_input_type_script_hash(data: &[u8]) -> Result<(), TestCaseError> {
.type_(Some(script).pack())
.build();
input_cell.cell_output = output_with_type;
- let outputs = Arc::new(vec![]);
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
@@ -1234,7 +1225,9 @@ fn _test_load_input_type_script_hash(data: &[u8]) -> Result<(), TestCaseError> {
resolved_dep_groups: vec![],
});
- let mut load_cell = LoadCell::new(data_loader, rtx, outputs, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_cell = LoadCell::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1279,8 +1272,6 @@ fn _test_load_witness(data: &[u8], source: SourceEntry) -> Result<(), TestCaseEr
let witness_correct_data = witness.raw_data();
let witnesses = vec![witness];
- let group_inputs = Arc::new(vec![]);
- let group_outputs = Arc::new(vec![]);
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
.witnesses(witnesses.pack())
@@ -1290,7 +1281,9 @@ fn _test_load_witness(data: &[u8], source: SourceEntry) -> Result<(), TestCaseEr
resolved_dep_groups: vec![],
});
- let mut load_witness = LoadWitness::new(rtx, group_inputs, group_outputs);
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let mut load_witness = LoadWitness::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1344,8 +1337,6 @@ fn _test_load_group_witness(data: &[u8], source: SourceEntry) -> Result<(), Test
let dummy_witness = Bytes::default().pack();
let witnesses = vec![dummy_witness, witness];
- let group_inputs = Arc::new(vec![1]);
- let group_outputs = Arc::new(vec![1]);
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
.witnesses(witnesses.pack())
@@ -1354,7 +1345,10 @@ fn _test_load_group_witness(data: &[u8], source: SourceEntry) -> Result<(), Test
resolved_inputs: vec![],
resolved_dep_groups: vec![],
});
- let mut load_witness = LoadWitness::new(rtx, group_inputs, group_outputs);
+
+ let sg_data = build_sg_data(rtx, vec![1], vec![1]);
+
+ let mut load_witness = LoadWitness::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1401,7 +1395,21 @@ fn _test_load_script(data: &[u8]) -> Result<(), TestCaseError> {
.build();
let script_correct_data = script.as_slice();
- let mut load_script = LoadScript::new(script.clone());
+ let rtx = Arc::new(ResolvedTransaction {
+ transaction: TransactionBuilder::default().build(),
+ resolved_cell_deps: vec![],
+ resolved_inputs: vec![],
+ resolved_dep_groups: vec![],
+ });
+
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+ // Swap the internal script in VmData
+ let sg_data = update_sg_info(sg_data, |sg_info| {
+ sg_info.script_hash = script.calc_script_hash();
+ sg_info.script_group.script = script.clone();
+ });
+
+ let mut load_script = LoadScript::new(&sg_data);
prop_assert!(machine
.memory_mut()
@@ -1455,7 +1463,6 @@ fn _test_load_cell_data_as_code(
let dep_cell = build_cell_meta(10000, data.clone());
let input_cell = build_cell_meta(100, data.clone());
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
.output_data(data.pack())
@@ -1464,17 +1471,12 @@ fn _test_load_cell_data_as_code(
resolved_inputs: vec![input_cell],
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: vec![0],
- output_indices: vec![0],
- }),
- }))));
+
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
prop_assert!(machine.memory_mut().store_byte(addr, addr_size, 1).is_ok());
@@ -1527,7 +1529,6 @@ fn _test_load_cell_data(
let data = Bytes::from(data.to_owned());
let dep_cell = build_cell_meta(10000, data.clone());
let input_cell = build_cell_meta(100, data.clone());
- let data_loader = new_mock_data_loader();
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
@@ -1537,17 +1538,12 @@ fn _test_load_cell_data(
resolved_inputs: vec![input_cell],
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: vec![0],
- output_indices: vec![0],
- }),
- }))));
+
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
prop_assert!(load_code.ecall(&mut machine).is_ok());
@@ -1638,8 +1634,6 @@ fn test_load_overflowed_cell_data_as_code() {
let dep_cell_data = Bytes::from(data);
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1647,17 +1641,11 @@ fn test_load_overflowed_cell_data_as_code() {
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
assert!(machine.memory_mut().store_byte(addr, addr_size, 1).is_ok());
@@ -1690,8 +1678,6 @@ fn _test_load_cell_data_on_freezed_memory(data: &[u8]) -> Result<(), TestCaseErr
let dep_cell_data = Bytes::from(data.to_owned());
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1699,17 +1685,11 @@ fn _test_load_cell_data_on_freezed_memory(data: &[u8]) -> Result<(), TestCaseErr
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
prop_assert!(load_code.ecall(&mut machine).is_err());
@@ -1740,8 +1720,6 @@ fn _test_load_cell_data_as_code_on_freezed_memory(data: &[u8]) -> Result<(), Tes
let dep_cell_data = Bytes::from(data.to_owned());
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1749,17 +1727,11 @@ fn _test_load_cell_data_as_code_on_freezed_memory(data: &[u8]) -> Result<(), Tes
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
prop_assert!(load_code.ecall(&mut machine).is_err());
@@ -1801,8 +1773,6 @@ fn test_load_code_unaligned_error() {
let dep_cell_data = Bytes::from(data.to_vec());
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1810,17 +1780,11 @@ fn test_load_code_unaligned_error() {
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
assert!(machine.memory_mut().store_byte(addr, addr_size, 1).is_ok());
@@ -1849,8 +1813,6 @@ fn test_load_code_slice_out_of_bound_error() {
let dep_cell_data = Bytes::from(data.to_vec());
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1858,17 +1820,11 @@ fn test_load_code_slice_out_of_bound_error() {
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
assert!(machine.memory_mut().store_byte(addr, addr_size, 1).is_ok());
@@ -1900,8 +1856,6 @@ fn test_load_code_not_enough_space_error() {
let dep_cell_data = Bytes::from(data);
let dep_cell = build_cell_meta(10000, dep_cell_data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
@@ -1909,17 +1863,11 @@ fn test_load_code_not_enough_space_error() {
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: Default::default(),
- output_indices: Default::default(),
- }),
- }))));
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
assert!(machine.memory_mut().store_byte(addr, addr_size, 1).is_ok());
@@ -1980,8 +1928,10 @@ fn _test_load_input(
resolved_inputs: vec![],
resolved_dep_groups: vec![],
});
- let group_inputs = Arc::new(vec![0]);
- let mut load_input = LoadInput::new(rtx, group_inputs);
+
+ let sg_data = build_sg_data(rtx, vec![0], vec![]);
+
+ let mut load_input = LoadInput::new(&sg_data);
let mut buffer = vec![];
let expect = if let Some(field) = field {
@@ -2111,25 +2061,19 @@ fn test_load_cell_data_size_zero() {
let dep_cell = build_cell_meta(10000, data.clone());
let input_cell = build_cell_meta(100, data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
resolved_inputs: vec![input_cell],
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: vec![0],
- output_indices: vec![0],
- }),
- }))));
+
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
+
load_code.ecall(&mut machine).unwrap();
assert_eq!(machine.registers()[A0], u64::from(SUCCESS));
assert_eq!(machine.memory_mut().load64(&size_addr).unwrap(), 256);
@@ -2156,25 +2100,19 @@ fn test_load_cell_data_size_zero_index_out_of_bound() {
let dep_cell = build_cell_meta(10000, data.clone());
let input_cell = build_cell_meta(100, data);
- let data_loader = new_mock_data_loader();
-
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default().build(),
resolved_cell_deps: vec![dep_cell],
resolved_inputs: vec![input_cell],
resolved_dep_groups: vec![],
});
- let mut load_code = LoadCellData::new(Arc::new(Mutex::new(Snapshot2Context::new(TxData {
- rtx,
- data_loader,
- program: Bytes::new(),
- script_group: Arc::new(ScriptGroup {
- script: Default::default(),
- group_type: ScriptGroupType::Lock,
- input_indices: vec![0],
- output_indices: vec![0],
- }),
- }))));
+
+ let sg_data = build_sg_data(rtx, vec![0], vec![0]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let mut load_code = LoadCellData::new(&vm_context);
+
load_code.ecall(&mut machine).unwrap();
assert_eq!(machine.registers()[A0], u64::from(INDEX_OUT_OF_BOUND));
}
diff --git a/script/src/syscalls/tests/vm_latest/syscalls_2.rs b/script/src/syscalls/tests/vm_latest/syscalls_2.rs
index 4cac872f56..0f40dc1e51 100644
--- a/script/src/syscalls/tests/vm_latest/syscalls_2.rs
+++ b/script/src/syscalls/tests/vm_latest/syscalls_2.rs
@@ -1,4 +1,4 @@
-use crate::syscalls::tests::utils::MockDataLoader;
+use crate::{syscalls::tests::utils::*, types::VmContext};
use ckb_types::{
bytes::Bytes,
core::{
@@ -53,7 +53,18 @@ fn test_current_cycles() {
machine.set_cycles(cycles);
- let result = CurrentCycles::new(Arc::new(Mutex::new(0))).ecall(&mut machine);
+ let rtx = Arc::new(ResolvedTransaction {
+ transaction: TransactionBuilder::default().build(),
+ resolved_cell_deps: vec![],
+ resolved_inputs: vec![],
+ resolved_dep_groups: vec![],
+ });
+
+ let sg_data = build_sg_data(rtx, vec![], vec![]);
+
+ let vm_context = VmContext::new(&sg_data, &Arc::new(Mutex::new(Vec::new())));
+
+ let result = CurrentCycles::new(&vm_context).ecall(&mut machine);
assert!(result.unwrap());
assert_eq!(machine.registers()[A0], cycles);
@@ -99,7 +110,6 @@ fn _test_load_extension(
extensions,
..Default::default()
};
- let group_inputs = Arc::new(vec![0]);
let rtx = Arc::new(ResolvedTransaction {
transaction: TransactionBuilder::default()
@@ -110,8 +120,9 @@ fn _test_load_extension(
resolved_dep_groups: vec![],
});
- let mut load_block_extension: LoadBlockExtension =
- LoadBlockExtension::new(data_loader, rtx, group_inputs);
+ let sg_data = build_sg_data_with_loader(rtx, data_loader, vec![0], vec![]);
+
+ let mut load_block_extension = LoadBlockExtension::new(&sg_data);
prop_assert!(machine
.memory_mut()
diff --git a/script/src/syscalls/wait.rs b/script/src/syscalls/wait.rs
index c9a49566aa..11ae3693e3 100644
--- a/script/src/syscalls/wait.rs
+++ b/script/src/syscalls/wait.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{SPAWN_YIELD_CYCLES_BASE, WAIT};
-use crate::types::{Message, VmId, WaitArgs};
+use crate::types::{Message, VmContext, VmId, WaitArgs};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A1, A7},
Error as VMError, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct Wait {
}
impl Wait {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/syscalls/write.rs b/script/src/syscalls/write.rs
index a193f5589b..2784abbeb4 100644
--- a/script/src/syscalls/write.rs
+++ b/script/src/syscalls/write.rs
@@ -1,5 +1,6 @@
use crate::syscalls::{INVALID_FD, SPAWN_YIELD_CYCLES_BASE, WRITE};
-use crate::types::{Fd, FdArgs, Message, VmId};
+use crate::types::{Fd, FdArgs, Message, VmContext, VmId};
+use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
use ckb_vm::{
registers::{A0, A1, A2, A7},
Error as VMError, Memory, Register, SupportMachine, Syscalls,
@@ -13,8 +14,14 @@ pub struct Write {
}
impl Write {
- pub fn new(id: VmId, message_box: Arc>>) -> Self {
- Self { id, message_box }
+ pub fn new(vm_id: &VmId, vm_context: &VmContext) -> Self
+ where
+ DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
+ {
+ Self {
+ id: *vm_id,
+ message_box: Arc::clone(&vm_context.message_box),
+ }
}
}
diff --git a/script/src/types.rs b/script/src/types.rs
index 8f25a98799..a7c9cdfbac 100644
--- a/script/src/types.rs
+++ b/script/src/types.rs
@@ -1,14 +1,24 @@
+use crate::{error::ScriptError, verify_env::TxVerifyEnv};
+use ckb_chain_spec::consensus::Consensus;
use ckb_types::{
- core::{Cycle, ScriptHashType},
- packed::{Byte32, Script},
+ core::{
+ cell::{CellMeta, ResolvedTransaction},
+ Cycle, ScriptHashType,
+ },
+ packed::{Byte32, CellOutput, OutPoint, Script},
+ prelude::*,
};
use ckb_vm::{
machine::{VERSION0, VERSION1, VERSION2},
ISA_B, ISA_IMC, ISA_MOP,
};
use serde::{Deserialize, Serialize};
+use std::collections::{BTreeMap, HashMap};
use std::fmt;
-use std::sync::{Arc, Mutex};
+use std::sync::{
+ atomic::{AtomicU64, Ordering},
+ Arc, Mutex, RwLock,
+};
#[cfg(has_asm)]
use ckb_vm::machine::asm::{AsmCoreMachine, AsmMachine};
@@ -16,10 +26,9 @@ use ckb_vm::machine::asm::{AsmCoreMachine, AsmMachine};
#[cfg(not(has_asm))]
use ckb_vm::{DefaultCoreMachine, TraceMachine, WXorXMemory};
-use ckb_traits::{CellDataProvider, ExtensionProvider, HeaderProvider};
+use ckb_traits::CellDataProvider;
use ckb_vm::snapshot2::Snapshot2Context;
-use ckb_types::core::cell::ResolvedTransaction;
use ckb_vm::{
bytes::Bytes,
machine::Pause,
@@ -54,10 +63,14 @@ pub(crate) type Machine = AsmMachine;
#[cfg(not(has_asm))]
pub(crate) type Machine = TraceMachine;
-pub(crate) type Indices = Arc>;
-
pub(crate) type DebugPrinter = Arc;
+pub struct DebugContext {
+ pub debug_printer: DebugPrinter,
+ #[cfg(test)]
+ pub skip_pause: Arc,
+}
+
/// The version of CKB Script Verifier.
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)]
pub enum ScriptVersion {
@@ -126,7 +139,7 @@ impl ScriptVersion {
/// A script group will only be executed once per transaction, the
/// script itself should check against all inputs/outputs in its group
/// if needed.
-#[derive(Clone)]
+#[derive(Clone, Debug)]
pub struct ScriptGroup {
/// The script.
///
@@ -140,9 +153,19 @@ pub struct ScriptGroup {
pub output_indices: Vec,
}
+/// The methods included here are defected in a way: all construction
+/// methods here create ScriptGroup without any `input_indices` or
+/// `output_indices` filled. One has to manually fill them later(or forgot
+/// about this).
+/// As a result, we are marking them as crate-only methods for now. This
+/// forces users to one of the following 2 solutions:
+/// * Call `groups()` on `TxData` so they can fetch `ScriptGroup` data with
+/// all correct data filled.
+/// * Manually construct the struct where they have to think what shall be
+/// used for `input_indices` and `output_indices`.
impl ScriptGroup {
/// Creates a new script group struct.
- pub fn new(script: &Script, group_type: ScriptGroupType) -> Self {
+ pub(crate) fn new(script: &Script, group_type: ScriptGroupType) -> Self {
Self {
group_type,
script: script.to_owned(),
@@ -152,12 +175,12 @@ impl ScriptGroup {
}
/// Creates a lock script group.
- pub fn from_lock_script(script: &Script) -> Self {
+ pub(crate) fn from_lock_script(script: &Script) -> Self {
Self::new(script, ScriptGroupType::Lock)
}
/// Creates a type script group.
- pub fn from_type_script(script: &Script) -> Self {
+ pub(crate) fn from_type_script(script: &Script) -> Self {
Self::new(script, ScriptGroupType::Type)
}
}
@@ -186,6 +209,7 @@ impl fmt::Display for ScriptGroupType {
/// Struct specifies which script has verified so far.
/// State is lifetime free, but capture snapshot need heavy memory copy
+#[derive(Clone)]
pub struct TransactionState {
/// current suspended script index
pub current: usize,
@@ -257,34 +281,6 @@ pub enum ChunkCommand {
Stop,
}
-#[derive(Clone)]
-pub struct MachineContext<
- DL: CellDataProvider + HeaderProvider + ExtensionProvider + Send + Sync + Clone + 'static,
-> {
- pub(crate) base_cycles: Arc>,
- pub(crate) snapshot2_context: Arc>>>,
-}
-
-impl MachineContext