Skip to content

Commit

Permalink
bpf_loader: use an explicit thread-local pool for stack and heap memory
Browse files Browse the repository at this point in the history
Use a fixed thread-local pool to hold stack and heap memory. This
mitigates the long standing issue of jemalloc causing TLB shootdowns to
serve such frequent large allocations.

Because we need 1 stack and 1 heap region per instruction, and the
current max instruction nesting is hardcoded to 5, the pre-allocated
size is (MAX_STACK + MAX_HEAP) * 5 * NUM_THREADS. With the current
limits that's about 2.5MB per thread. Note that this is memory that
would eventually get allocated anyway, we're just pre-allocating it now.
  • Loading branch information
alessandrod committed Jun 10, 2024
1 parent b716121 commit 52eb624
Show file tree
Hide file tree
Showing 7 changed files with 190 additions and 32 deletions.
14 changes: 8 additions & 6 deletions compute-budget/src/compute_budget.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,14 +13,16 @@ impl ::solana_frozen_abi::abi_example::AbiExample for ComputeBudget {
}
}

/// Roughly 0.5us/page, where page is 32K; given roughly 15CU/us, the
/// default heap page cost = 0.5 * 15 ~= 8CU/page
pub const DEFAULT_HEAP_COST: u64 = 8;

/// Max instruction stack depth. This is the maximum nesting of instructions that can happen during
/// a transaction.
pub const MAX_INSTRUCTION_STACK_DEPTH: usize = 5;

/// Max call depth. This is the maximum nesting of SBF to SBF call that can happen within a program.
pub const MAX_CALL_DEPTH: usize = 64;

/// The size of one SBF stack frame.
pub const STACK_FRAME_SIZE: usize = 4096;

#[derive(Clone, Copy, Debug, PartialEq, Eq)]
pub struct ComputeBudget {
/// Number of compute units that a transaction or individual instruction is
Expand Down Expand Up @@ -146,8 +148,8 @@ impl ComputeBudget {
sha256_base_cost: 85,
sha256_byte_cost: 1,
sha256_max_slices: 20_000,
max_call_depth: 64,
stack_frame_size: 4_096,
max_call_depth: MAX_CALL_DEPTH,
stack_frame_size: STACK_FRAME_SIZE,
log_pubkey_units: 100,
max_cpi_instruction_size: 1280, // IPv6 Min MTU size
cpi_bytes_per_unit: 250, // ~50MB at 200,000 units
Expand Down
7 changes: 4 additions & 3 deletions compute-budget/src/compute_budget_processor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,20 +3,21 @@ use {
solana_sdk::{
borsh1::try_from_slice_unchecked,
compute_budget::{self, ComputeBudgetInstruction},
entrypoint::HEAP_LENGTH as MIN_HEAP_FRAME_BYTES,
entrypoint::HEAP_LENGTH,
fee::FeeBudgetLimits,
instruction::{CompiledInstruction, InstructionError},
pubkey::Pubkey,
transaction::TransactionError,
},
};

const MAX_HEAP_FRAME_BYTES: u32 = 256 * 1024;
/// Roughly 0.5us/page, where page is 32K; given roughly 15CU/us, the
/// default heap page cost = 0.5 * 15 ~= 8CU/page
pub const DEFAULT_HEAP_COST: u64 = 8;
pub const DEFAULT_INSTRUCTION_COMPUTE_UNIT_LIMIT: u32 = 200_000;
pub const MAX_COMPUTE_UNIT_LIMIT: u32 = 1_400_000;
pub const MAX_HEAP_FRAME_BYTES: u32 = 256 * 1024;
pub const MIN_HEAP_FRAME_BYTES: u32 = HEAP_LENGTH as u32;

/// The total accounts data a transaction can load is limited to 64MiB to not break
/// anyone in Mainnet-beta today. It can be set by set_loaded_accounts_data_size_limit instruction
Expand All @@ -33,7 +34,7 @@ pub struct ComputeBudgetLimits {
impl Default for ComputeBudgetLimits {
fn default() -> Self {
ComputeBudgetLimits {
updated_heap_bytes: u32::try_from(MIN_HEAP_FRAME_BYTES).unwrap(),
updated_heap_bytes: MIN_HEAP_FRAME_BYTES,
compute_unit_limit: MAX_COMPUTE_UNIT_LIMIT,
compute_unit_price: 0,
loaded_accounts_bytes: MAX_LOADED_ACCOUNTS_DATA_SIZE_BYTES,
Expand Down
2 changes: 1 addition & 1 deletion ledger-tool/src/program.rs
Original file line number Diff line number Diff line change
Expand Up @@ -578,7 +578,7 @@ pub fn program(ledger_path: &Path, matches: &ArgMatches<'_>) {
account_lengths,
&mut invoke_context,
);
let mut vm = vm.unwrap();
let (mut vm, _, _) = vm.unwrap();
let start_time = Instant::now();
if matches.value_of("mode").unwrap() == "debugger" {
vm.debug_port = Some(matches.value_of("port").unwrap().parse::<u16>().unwrap());
Expand Down
52 changes: 32 additions & 20 deletions programs/bpf_loader/src/lib.rs
Original file line number Diff line number Diff line change
@@ -1,10 +1,13 @@
#![deny(clippy::arithmetic_side_effects)]
#![deny(clippy::indexing_slicing)]

pub mod mem_pool;
pub mod serialization;
pub mod syscalls;

use {
mem_pool::VmMemoryPool,
solana_compute_budget::compute_budget::MAX_INSTRUCTION_STACK_DEPTH,
solana_measure::measure::Measure,
solana_program_runtime::{
ic_logger_msg, ic_msg,
Expand All @@ -18,9 +21,8 @@ use {
sysvar_cache::get_sysvar_with_account_check,
},
solana_rbpf::{
aligned_memory::AlignedMemory,
declare_builtin_function,
ebpf::{self, HOST_ALIGN, MM_HEAP_START},
ebpf::{self, MM_HEAP_START},
elf::Executable,
error::{EbpfError, ProgramResult},
memory_region::{AccessType, MemoryCowCallback, MemoryMapping, MemoryRegion},
Expand Down Expand Up @@ -59,6 +61,10 @@ pub const DEFAULT_LOADER_COMPUTE_UNITS: u64 = 570;
pub const DEPRECATED_LOADER_COMPUTE_UNITS: u64 = 1_140;
pub const UPGRADEABLE_LOADER_COMPUTE_UNITS: u64 = 2_370;

thread_local! {
pub static MEMORY_POOL: RefCell<VmMemoryPool> = RefCell::new(VmMemoryPool::new());
}

#[allow(clippy::too_many_arguments)]
pub fn load_program_from_bytes(
log_collector: Option<Rc<RefCell<LogCollector>>>,
Expand Down Expand Up @@ -244,8 +250,8 @@ pub fn create_vm<'a, 'b>(
regions: Vec<MemoryRegion>,
accounts_metadata: Vec<SerializedAccountMetadata>,
invoke_context: &'a mut InvokeContext<'b>,
stack: &mut AlignedMemory<HOST_ALIGN>,
heap: &mut AlignedMemory<HOST_ALIGN>,
stack: &mut [u8],
heap: &mut [u8],
) -> Result<EbpfVm<'a, InvokeContext<'b>>, Box<dyn std::error::Error>> {
let stack_size = stack.len();
let heap_size = heap.len();
Expand Down Expand Up @@ -299,24 +305,23 @@ macro_rules! create_vm {
heap_size,
invoke_context.get_compute_budget().heap_cost,
));
let mut allocations = None;
let $vm = heap_cost_result.and_then(|_| {
let mut stack = solana_rbpf::aligned_memory::AlignedMemory::<
{ solana_rbpf::ebpf::HOST_ALIGN },
>::zero_filled(stack_size);
let mut heap = solana_rbpf::aligned_memory::AlignedMemory::<
{ solana_rbpf::ebpf::HOST_ALIGN },
>::zero_filled(usize::try_from(heap_size).unwrap());
let (mut stack, mut heap) = $crate::MEMORY_POOL
.with_borrow_mut(|pool| (pool.get_stack(stack_size), pool.get_heap(heap_size)));
let vm = $crate::create_vm(
$program,
$regions,
$accounts_metadata,
$invoke_context,
&mut stack,
&mut heap,
stack
.as_slice_mut()
.get_mut(..stack_size)
.expect("invalid stack size"),
heap.as_slice_mut()
.get_mut(..heap_size as usize)
.expect("invalid heap size"),
);
allocations = Some((stack, heap));
vm
vm.map(|vm| (vm, stack, heap))
});
};
}
Expand All @@ -343,13 +348,14 @@ macro_rules! mock_create_vm {
$accounts_metadata,
$invoke_context,
);
let $vm = $vm.map(|(vm, _, _)| vm);
};
}

fn create_memory_mapping<'a, 'b, C: ContextObject>(
executable: &'a Executable<C>,
stack: &'b mut AlignedMemory<{ HOST_ALIGN }>,
heap: &'b mut AlignedMemory<{ HOST_ALIGN }>,
stack: &'b mut [u8],
heap: &'b mut [u8],
additional_regions: Vec<MemoryRegion>,
cow_cb: Option<MemoryCowCallback>,
) -> Result<MemoryMapping<'a>, Box<dyn std::error::Error>> {
Expand All @@ -358,15 +364,15 @@ fn create_memory_mapping<'a, 'b, C: ContextObject>(
let regions: Vec<MemoryRegion> = vec![
executable.get_ro_region(),
MemoryRegion::new_writable_gapped(
stack.as_slice_mut(),
stack,
ebpf::MM_STACK_START,
if !sbpf_version.dynamic_stack_frames() && config.enable_stack_frame_gaps {
config.stack_frame_size as u64
} else {
0
},
),
MemoryRegion::new_writable(heap.as_slice_mut(), MM_HEAP_START),
MemoryRegion::new_writable(heap, MM_HEAP_START),
]
.into_iter()
.chain(additional_regions)
Expand Down Expand Up @@ -1388,7 +1394,7 @@ fn execute<'a, 'b: 'a>(
let execution_result = {
let compute_meter_prev = invoke_context.get_remaining();
create_vm!(vm, executable, regions, accounts_metadata, invoke_context);
let mut vm = match vm {
let (mut vm, stack, heap) = match vm {
Ok(info) => info,
Err(e) => {
ic_logger_msg!(log_collector, "Failed to create SBF VM: {}", e);
Expand All @@ -1399,6 +1405,12 @@ fn execute<'a, 'b: 'a>(

vm.context_object_pointer.execute_time = Some(Measure::start("execute"));
let (compute_units_consumed, result) = vm.execute_program(executable, !use_jit);
MEMORY_POOL.with_borrow_mut(|memory_pool| {
memory_pool.put_stack(stack);
memory_pool.put_heap(heap);
debug_assert!(memory_pool.stack_len() <= MAX_INSTRUCTION_STACK_DEPTH);
debug_assert!(memory_pool.heap_len() <= MAX_INSTRUCTION_STACK_DEPTH);
});
drop(vm);
if let Some(execute_time) = invoke_context.execute_time.as_mut() {
execute_time.stop();
Expand Down
142 changes: 142 additions & 0 deletions programs/bpf_loader/src/mem_pool.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
use {
solana_compute_budget::{
compute_budget::{MAX_CALL_DEPTH, MAX_INSTRUCTION_STACK_DEPTH, STACK_FRAME_SIZE},
compute_budget_processor::{MAX_HEAP_FRAME_BYTES, MIN_HEAP_FRAME_BYTES},
},
solana_rbpf::{aligned_memory::AlignedMemory, ebpf::HOST_ALIGN},
std::array,
};

trait Reset {
fn reset(&mut self);
}

struct Pool<T: Reset, const SIZE: usize> {
items: [Option<T>; SIZE],
next_empty: usize,
}

impl<T: Reset, const SIZE: usize> Pool<T, SIZE> {
fn new(items: [T; SIZE]) -> Self {
Self {
items: items.map(|i| Some(i)),
next_empty: SIZE,
}
}

fn len(&self) -> usize {
SIZE
}

fn get(&mut self) -> Option<T> {
if self.next_empty == 0 {
return None;
}
self.next_empty -= 1;
self.items[self.next_empty].take()
}

fn put(&mut self, mut value: T) -> bool {
if self.next_empty == self.items.len() {
return false;
}
value.reset();
self.items[self.next_empty] = Some(value);
self.next_empty += 1;
true
}
}

impl Reset for AlignedMemory<{ HOST_ALIGN }> {
fn reset(&mut self) {
self.as_slice_mut().fill(0)
}
}

pub struct VmMemoryPool {
stack: Pool<AlignedMemory<{ HOST_ALIGN }>, MAX_INSTRUCTION_STACK_DEPTH>,
heap: Pool<AlignedMemory<{ HOST_ALIGN }>, MAX_INSTRUCTION_STACK_DEPTH>,
}

impl VmMemoryPool {
pub fn new() -> Self {
Self {
stack: Pool::new(array::from_fn(|_| {
AlignedMemory::zero_filled(STACK_FRAME_SIZE * MAX_CALL_DEPTH)
})),
heap: Pool::new(array::from_fn(|_| {
AlignedMemory::zero_filled(MAX_HEAP_FRAME_BYTES as usize)
})),
}
}

pub fn stack_len(&self) -> usize {
self.stack.len()
}

pub fn heap_len(&self) -> usize {
self.heap.len()
}

pub fn get_stack(&mut self, size: usize) -> AlignedMemory<{ HOST_ALIGN }> {
debug_assert!(size == STACK_FRAME_SIZE * MAX_CALL_DEPTH);
self.stack
.get()
.unwrap_or_else(|| AlignedMemory::zero_filled(size))
}

pub fn put_stack(&mut self, stack: AlignedMemory<{ HOST_ALIGN }>) -> bool {
self.stack.put(stack)
}

pub fn get_heap(&mut self, heap_size: u32) -> AlignedMemory<{ HOST_ALIGN }> {
debug_assert!(heap_size >= MIN_HEAP_FRAME_BYTES && heap_size <= MAX_HEAP_FRAME_BYTES);
self.heap
.get()
.unwrap_or_else(|| AlignedMemory::zero_filled(MAX_HEAP_FRAME_BYTES as usize))
}

pub fn put_heap(&mut self, heap: AlignedMemory<{ HOST_ALIGN }>) -> bool {
let heap_size = heap.len();
debug_assert!(
heap_size >= MIN_HEAP_FRAME_BYTES as usize
&& heap_size <= MAX_HEAP_FRAME_BYTES as usize
);
self.heap.put(heap)
}
}

impl Default for VmMemoryPool {
fn default() -> Self {
Self::new()
}
}

#[cfg(test)]
mod test {
use super::*;

#[derive(Debug, Eq, PartialEq)]
struct Item(u8, u8);
impl Reset for Item {
fn reset(&mut self) {
self.1 = 0;
}
}

#[test]
fn test_pool() {
let mut pool = Pool::<Item, 2>::new([Item(0, 1), Item(1, 1)]);
assert_eq!(pool.get(), Some(Item(1, 1)));
assert_eq!(pool.get(), Some(Item(0, 1)));
assert_eq!(pool.get(), None);
pool.put(Item(1, 1));
assert_eq!(pool.get(), Some(Item(1, 0)));
pool.put(Item(2, 2));
pool.put(Item(3, 3));
assert!(!pool.put(Item(4, 4)));
assert_eq!(pool.get(), Some(Item(3, 0)));
assert_eq!(pool.get(), Some(Item(2, 0)));
assert_eq!(pool.get(), None);
}
}
1 change: 1 addition & 0 deletions programs/bpf_loader/src/syscalls/cpi.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2539,6 +2539,7 @@ mod tests {

let config = Config {
aligned_memory_mapping: false,
max_call_depth: 64,
..Config::default()
};
let memory_mapping = MemoryMapping::new(vec![region], &config, &SBPFVersion::V2).unwrap();
Expand Down
4 changes: 2 additions & 2 deletions programs/sbf/benches/bpf_loader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ fn bench_program_alu(bencher: &mut Bencher) {
vec![],
&mut invoke_context,
);
let mut vm = vm.unwrap();
let (mut vm, _, _) = vm.unwrap();

println!("Interpreted:");
vm.context_object_pointer
Expand Down Expand Up @@ -314,7 +314,7 @@ fn bench_instruction_count_tuner(_bencher: &mut Bencher) {
account_lengths,
&mut invoke_context,
);
let mut vm = vm.unwrap();
let (mut vm, _, _) = vm.unwrap();

let mut measure = Measure::start("tune");
let (instructions, _result) = vm.execute_program(&executable, true);
Expand Down

0 comments on commit 52eb624

Please sign in to comment.