rv64: smp secondary hart startup

This commit is contained in:
Mark Poliakov 2025-01-20 17:04:17 +02:00
parent 65b8c0ee67
commit 16f580e7af
10 changed files with 340 additions and 185 deletions

View File

@ -3,11 +3,15 @@
extern crate alloc;
use alloc::{boxed::Box, vec::Vec};
use core::sync::atomic::{AtomicUsize, Ordering};
use alloc::{boxed::Box, collections::btree_map::BTreeMap, vec::Vec};
use device_api::interrupt::{LocalInterruptController, MessageInterruptController};
use kernel_arch_interface::{
cpu::{CpuData, CpuImpl, IpiQueue},
sync::IrqSafeSpinlock,
task::Scheduler,
util::OneTimeInit,
Architecture,
};
use tock_registers::interfaces::{ReadWriteable, Readable};
@ -36,6 +40,11 @@ pub struct PerCpuData {
pub queue_index: usize,
}
pub static CPU_COUNT: AtomicUsize = AtomicUsize::new(1);
static IPI_QUEUES: OneTimeInit<Vec<IpiQueue<ArchitectureImpl>>> = OneTimeInit::new();
static HART_TO_QUEUE: IrqSafeSpinlock<ArchitectureImpl, BTreeMap<u32, usize>> =
IrqSafeSpinlock::new(BTreeMap::new());
impl CpuData for PerCpuData {
fn is_bootstrap(&self, id: u32) -> bool {
let _ = id;
@ -55,6 +64,16 @@ extern "C" fn idle_task(_: usize) -> ! {
}
}
impl ArchitectureImpl {
pub fn for_each_hart<F: FnMut(u32, usize, &IpiQueue<ArchitectureImpl>)>(mut f: F) {
let map = HART_TO_QUEUE.lock();
map.iter().for_each(|(&hart_id, &queue_index)| {
let queue = &IPI_QUEUES.get()[queue_index];
f(hart_id, queue_index, queue);
});
}
}
impl Architecture for ArchitectureImpl {
type PerCpuData = PerCpuData;
type CpuFeatures = ();
@ -82,19 +101,19 @@ impl Architecture for ArchitectureImpl {
unsafe fn init_local_cpu<S: Scheduler + 'static>(id: Option<u32>, data: Self::PerCpuData) {
let id = id.expect("riscv64 requires an explicit HART ID in its per-processor struct");
let queue_index = data.queue_index;
HART_TO_QUEUE.lock().insert(id, queue_index);
let cpu = Box::leak(Box::new(CpuImpl::<Self, S>::new(id, data)));
unsafe { cpu.set_local() };
}
unsafe fn init_ipi_queues(queues: Vec<IpiQueue<Self>>) {
// TODO
let _ = queues;
// loop {}
IPI_QUEUES.init(queues);
}
fn ipi_queue(cpu_id: u32) -> Option<&'static IpiQueue<Self>> {
let _ = cpu_id;
todo!()
let queue_index = *HART_TO_QUEUE.lock().get(&cpu_id)?;
IPI_QUEUES.try_get().and_then(|q| q.get(queue_index))
}
#[inline]
@ -120,8 +139,7 @@ impl Architecture for ArchitectureImpl {
}
fn cpu_count() -> usize {
// TODO
1
CPU_COUNT.load(Ordering::Acquire)
}
fn cpu_index<S: Scheduler + 'static>() -> u32 {

View File

@ -4,7 +4,7 @@ use kernel_arch_interface::{
};
use libk_mm_interface::{
address::PhysicalAddress,
table::{page_index, EntryLevel, EntryLevelExt},
table::{page_align_down, page_index, EntryLevel, EntryLevelExt},
};
use memtables::riscv64::PageAttributes;
use static_assertions::{const_assert, const_assert_eq};
@ -240,21 +240,28 @@ pub unsafe fn enable_mmu() {
SATP.write(SATP::PPN.val(l1_phys >> 12) + SATP::MODE::Sv39);
}
/// Sets up run-time kernel translation tables and removed the lower-half mapping.
/// Removes the lower half translation mappings.
///
/// # Safety
///
/// The caller must ensure MMU is already enabled and that lower-half addresses will no
/// longer be referred to.
pub unsafe fn setup_fixed_tables() {
/// Needs to be called once after secondary HARTs are initialized.
pub unsafe fn unmap_lower_half() {
let mut tables = KERNEL_TABLES.lock();
let kernel_l1i_lower = page_index::<L1>(KERNEL_PHYS_BASE);
tables.l1.data[kernel_l1i_lower] = 0;
tlb_flush_va(page_align_down::<L1>(KERNEL_PHYS_BASE));
}
/// Sets up run-time kernel translation tables.
///
/// # Safety
///
/// The caller must ensure MMU is already enabled.
pub unsafe fn setup_fixed_tables() {
let mut tables = KERNEL_TABLES.lock();
let device_mapping_l2_phys = auto_address(&raw const DEVICE_MAPPING_L2);
// Unmap the lower half
tables.l1.data[kernel_l1i_lower] = 0;
// Set up static runtime mappings
for i in 0..DEVICE_MAPPING_L3_COUNT {
unsafe {

View File

@ -1,3 +1,55 @@
use yggdrasil_abi::{error::Error, primitive_enum};
const EXT_HSM: u64 = 0x48534D;
const EXT_TIME: u64 = 0x54494D45;
const EXT_DBCN: u64 = 0x4442434E;
const EXT_SPI: u64 = 0x735049;
primitive_enum! {
pub enum Status: i64 {
Failed = -1,
NotSupported = -2,
InvalidParam = -3,
Denied = -4,
InvalidAddress = -5,
AlreadyAvailable = -6,
AlreadyStarted = -7,
AlreadyStopped = -8,
NoShmem = -9,
InvalidState = -10,
BadRange = -11,
Timeout = -12,
Io = -13,
}
}
primitive_enum! {
pub enum HartState: u64 {
Started = 0,
Stopped = 1,
StartPending = 2,
StopPending = 3,
Suspended = 4,
SuspendPending = 5,
ResumePending = 6,
}
}
pub enum SbiError {
Status(Status),
Other(i64),
}
impl From<i64> for SbiError {
#[inline]
fn from(value: i64) -> Self {
match Status::try_from(value) {
Ok(value) => Self::Status(value),
Err(_) => Self::Other(value),
}
}
}
#[allow(clippy::too_many_arguments)]
#[inline(always)]
unsafe fn sbi_do_call(
@ -9,7 +61,7 @@ unsafe fn sbi_do_call(
a3: u64,
a4: u64,
a5: u64,
) {
) -> Result<u64, SbiError> {
unsafe {
core::arch::asm!(
"ecall",
@ -23,15 +75,36 @@ unsafe fn sbi_do_call(
in("a7") extension,
);
}
// TODO return `struct sbiret`
let _ = a0;
let _ = a1;
let a0 = a0 as i64;
if a0 == 0 {
Ok(a1)
} else {
Err(a0.into())
}
}
pub fn sbi_hart_start(hart_id: u64, start_addr: u64, opaque: u64) -> Result<(), Error> {
match unsafe { sbi_do_call(EXT_HSM, 0x00, hart_id, start_addr, opaque, 0, 0, 0) } {
Ok(_) => Ok(()),
Err(SbiError::Status(Status::AlreadyAvailable)) => Err(Error::AlreadyExists),
Err(SbiError::Status(Status::InvalidParam)) => Err(Error::DoesNotExist),
Err(SbiError::Status(Status::InvalidAddress)) => Err(Error::InvalidArgument),
Err(_) => Err(Error::InvalidOperation),
}
}
pub fn sbi_send_ipi(hart_mask: u64, hart_mask_base: u64) -> Result<(), Error> {
match unsafe { sbi_do_call(EXT_SPI, 0x00, hart_mask, hart_mask_base, 0, 0, 0, 0) } {
Ok(_) => Ok(()),
Err(SbiError::Status(Status::InvalidParam)) => Err(Error::DoesNotExist),
Err(_) => Err(Error::InvalidOperation),
}
}
pub fn sbi_debug_console_write_byte(byte: u8) {
unsafe { sbi_do_call(0x4442434E, 0x02, byte as u64, 0, 0, 0, 0, 0) };
unsafe { sbi_do_call(EXT_DBCN, 0x02, byte as u64, 0, 0, 0, 0, 0) }.ok();
}
pub fn sbi_set_timer(next_event: u64) {
unsafe { sbi_do_call(0x54494D45, 0x00, next_event, 0, 0, 0, 0, 0) };
unsafe { sbi_do_call(EXT_TIME, 0x00, next_event, 0, 0, 0, 0, 0) }.ok();
}

View File

@ -25,6 +25,7 @@ use libk_util::{
use yggdrasil_abi::error::Error;
use crate::{
arch::Cpu,
config,
fs::sysfs::{
self,
@ -158,6 +159,7 @@ impl log::Log for DebugSinkWrapper {
let level = LogLevel::from(record.level());
let sink = self.sink();
let cpu = Cpu::try_local().map(|c| c.id());
let file = record.file().unwrap_or("<???>");
let line = record.line().unwrap_or(0);
let args = record.args();
@ -181,7 +183,13 @@ impl log::Log for DebugSinkWrapper {
writeln!(writer, "[io] {args}").ok();
}
_ => {
writeln!(writer, "{prefix}{file}:{line}: {args}{suffix}").ok();
write!(writer, "{prefix}").ok();
if let Some(cpu) = cpu {
write!(writer, "{cpu}:").ok();
} else {
write!(writer, "?:").ok();
}
writeln!(writer, "{file}:{line}: {args}{suffix}").ok();
}
}
}

View File

@ -5,16 +5,19 @@
addi \register, \register, %pcrel_lo(\label)
.endm
.section .text.entry
.pushsection .text.entry
.option norvc
.global __rv64_entry
.global __rv64_secondary_entry
.type __rv64_entry, @function
__rv64_entry:
// a0 - bootstrap HART ID
// a1 - device tree blob
// mhartid == a0
csrw satp, zero
mv tp, zero
// Zero the .bss
LOAD_PCREL .L00, t0, __bss_start_phys
@ -32,58 +35,19 @@ __rv64_entry:
jr t0
.size __rv64_entry, . - __rv64_entry
// .section .text.entry
// .option norvc
// .type __rv64_entry, @function
// .global __rv64_entry
// __rv64_entry:
// // Jump to parking place if hard id is not zero
// csrr t0, mhartid
// bnez t0, .spin_loop
//
// // Reset translation control
// csrw satp, zero
//
// // Zero the .bss
// LOAD_PCREL .L00, t0, __bss_start_phys
// LOAD_PCREL .L01, t1, __bss_end_phys
//
// 1: bgeu t0, t1, 2f
// sd zero, (t0)
// addi t0, t0, 4
// j 1b
// 2:
//
// // Setup boot stack
// LOAD_PCREL .L02, sp, {boot_stack_bottom} + {boot_stack_size} - {kernel_virt_offset}
//
// // Jump to entry
// LOAD_PCREL .L03, t0, entry_mmode_lower - {kernel_virt_offset}
//
// mv a0, a1
// jr t0
//
// 3: wfi
// j 3b
//
// .spin_loop:
// wfi
// j .spin_loop
//
// .size __rv64_entry, . - __rv64_entry
// .section .text
// .global __rv64_smode_entry
// .type __rv64_smode_entry, @function
// .p2align 4
// __rv64_smode_entry:
// // Set up the stack again
// LOAD_PCREL .L04, sp, {boot_stack_bottom} + {boot_stack_size}
// // Enter kernel proper
// LOAD_PCREL .L05, t0, {entry_smode_lower}
//
// jr t0
//
// 1: wfi
// j 1b
// .size __rv64_smode_entry, . - __rv64_smode_entry
.type __rv64_secondary_entry, @function
__rv64_secondary_entry:
// a1 - context struct
csrw satp, zero
mv tp, zero
// Setup stack and jump to Rust entry code
ld sp, (a1)
mv a0, a1
mv a1, sp
LOAD_PCREL .L04, t0, {entry_smode_secondary_lower} - {kernel_virt_offset}
jr t0
.size __rv64_secondary_entry, . - __rv64_secondary_entry
.popsection

View File

@ -1,20 +1,26 @@
use core::arch::global_asm;
use core::{
arch::global_asm,
sync::atomic::{compiler_fence, Ordering},
};
use kernel_arch::Architecture;
use kernel_arch_riscv64::{
mem::{self, KERNEL_VIRT_OFFSET},
ArchitectureImpl,
ArchitectureImpl, CPU_COUNT,
};
use libk::{
debug,
fs::{devfs, sysfs},
task::runtime,
};
use libk_mm::address::PhysicalAddress;
use libk_mm::{
address::{PhysicalAddress, Virtualize},
pointer::PhysicalRef,
};
use crate::kernel_main;
use crate::{kernel_main, kernel_secondary_main};
use super::PLATFORM;
use super::{smp::SecondaryContext, PLATFORM};
const BOOT_STACK_SIZE: usize = 65536;
@ -30,8 +36,6 @@ impl<const N: usize> BootStack<N> {
#[link_section = ".bss"]
static mut BOOT_STACK: BootStack<BOOT_STACK_SIZE> = BootStack::zeroed();
// static mut DTB_PHYSICAL: PhysicalAddress = PhysicalAddress::ZERO;
unsafe fn long_jump(pc: usize, sp: usize, a0: usize, a1: usize) -> ! {
core::arch::asm!(r#"
mv sp, {sp}
@ -45,18 +49,6 @@ unsafe fn long_jump(pc: usize, sp: usize, a0: usize, a1: usize) -> ! {
);
}
unsafe extern "C" fn __rv64_bsp_smode_entry_lower(a0: usize, a1: usize) -> ! {
ArchitectureImpl::set_interrupt_mask(true);
mem::enable_mmu();
let stack = (&raw const BOOT_STACK).addr() + KERNEL_VIRT_OFFSET;
let pc = __rv64_bsp_entry_upper as usize + KERNEL_VIRT_OFFSET;
let sp = stack + BOOT_STACK_SIZE;
long_jump(pc, sp, a0, a1)
}
unsafe extern "C" fn __rv64_bsp_entry_upper(bsp_hart_id: u64, dtb_physical: PhysicalAddress) -> ! {
debug::init_logger();
super::debug::register_sbi_debug();
@ -79,7 +71,7 @@ unsafe extern "C" fn __rv64_bsp_entry_upper(bsp_hart_id: u64, dtb_physical: Phys
runtime::init_task_queue();
if let Err(error) = PLATFORM.init_platform(bsp_hart_id as _, true) {
if let Err(error) = PLATFORM.init_platform(bsp_hart_id as _, 0, true) {
log::error!("Failed to initialize the platform: {error:?}");
ArchitectureImpl::halt();
}
@ -87,81 +79,50 @@ unsafe extern "C" fn __rv64_bsp_entry_upper(bsp_hart_id: u64, dtb_physical: Phys
kernel_main()
}
// // Drop to S-mode
// unsafe extern "C" fn __rv64_bsp_mmode_entry_lower(dtb: PhysicalAddress) -> ! {
// extern "C" {
// fn __rv64_smode_entry() -> !;
// }
//
// // Set mscratch to M-mode trap stack
// let trap_stack_bottom = (&raw const BSP_MMODE_TRAP_STACK).addr();
// let trap_sp = trap_stack_bottom + BOOT_STACK_SIZE;
// MSCRATCH.set(trap_sp as u64);
//
// // Setup trap vector for M-mode
// exception::init_mmode_exceptions();
//
// // Setup trap delegation to S-mode:
// // * S-mode timer -> S-mode
// // * All exceptions, except ecall from S-mode
// MIDELEG.modify(MIE::STIE::SET);
// MEDELEG.set(u64::MAX);
// MEDELEG.modify(MEDELEG::ECALL_SMODE::CLEAR);
//
// MCOUNTEREN.modify(MCOUNTEREN::CY::SET + MCOUNTEREN::TM::SET + MCOUNTEREN::IR::SET);
//
// MSTATUS.modify(
// // Mask S-mode interrupts
// MSTATUS::SIE::SET
// + MSTATUS::MPIE::CLEAR
// // UXLEN=SXLEN=64
// + MSTATUS::UXL.val(2)
// + MSTATUS::SXL.val(2)
// // Little endian
// + MSTATUS::UBE::CLEAR
// + MSTATUS::SBE::CLEAR
// // Don't trap S-mode VM insns, sret + U-mode wfi
// + MSTATUS::TVM::CLEAR
// + MSTATUS::TW::CLEAR
// + MSTATUS::TSR::CLEAR
// // Disable effective privilege modification
// + MSTATUS::MPRV::CLEAR
// // Enable S-mode access to U-mode pages
// + MSTATUS::SUM::SET
// // Make mret return to S-mode
// + MSTATUS::MPP::S,
// );
//
// MIE.modify(MIE::STIE::SET + MIE::SEIE::SET + MIE::MTIE::SET);
//
// let entry = __rv64_smode_entry as usize - KERNEL_VIRT_OFFSET;
// MEPC.set(entry as u64);
//
// // Modify pmpcfg/pmpaddr to allow lower-level execution
// unsafe {
// let mut pmpcfg0: u64;
// core::arch::asm!("csrr {0}, pmpcfg0", out(reg) pmpcfg0);
//
// let pmpaddr0: u64 = 0xFFFFffffFFFFffff;
//
// pmpcfg0 &= !0xFF;
// // A = 1, X, W, R
// pmpcfg0 |= 0xF;
//
// core::arch::asm!("csrw pmpaddr0, {0}; csrw pmpcfg0, {1}", in(reg) pmpaddr0, in(reg) pmpcfg0);
// }
//
// // Store the DTB address
// unsafe {
// DTB_PHYSICAL = dtb;
// }
//
// core::arch::asm!("mret", options(noreturn));
// }
unsafe extern "C" fn __rv64_secondary_entry_upper(context: PhysicalAddress) -> ! {
let context = PhysicalRef::<SecondaryContext>::map(context);
let queue_index = CPU_COUNT.fetch_add(1, Ordering::Acquire);
if let Err(error) = PLATFORM.init_platform(context.hart_id as u32, queue_index, false) {
log::error!("Secondary hart init error: {error:?}");
ArchitectureImpl::halt();
}
kernel_secondary_main()
}
unsafe extern "C" fn __rv64_bsp_smode_entry_lower(a0: usize, a1: usize) -> ! {
ArchitectureImpl::set_interrupt_mask(true);
mem::enable_mmu();
let stack = (&raw const BOOT_STACK).addr() + KERNEL_VIRT_OFFSET;
let pc = __rv64_bsp_entry_upper as usize + KERNEL_VIRT_OFFSET;
let sp = stack + BOOT_STACK_SIZE;
long_jump(pc, sp, a0, a1)
}
unsafe extern "C" fn __rv64_secondary_smode_entry_lower(
context: PhysicalAddress,
sp: PhysicalAddress,
) -> ! {
let sp = sp.virtualize();
ArchitectureImpl::set_interrupt_mask(true);
compiler_fence(Ordering::Release);
mem::enable_mmu();
compiler_fence(Ordering::Acquire);
let pc = __rv64_secondary_entry_upper as usize + KERNEL_VIRT_OFFSET;
long_jump(pc, sp, context.into_usize(), 0)
}
global_asm!(
include_str!("entry.S"),
entry_smode_lower = sym __rv64_bsp_smode_entry_lower,
entry_smode_secondary_lower = sym __rv64_secondary_smode_entry_lower,
boot_stack_bottom = sym BOOT_STACK,
kernel_virt_offset = const KERNEL_VIRT_OFFSET,
boot_stack_size = const BOOT_STACK_SIZE,

View File

@ -1,15 +1,15 @@
use core::arch::global_asm;
use abi::{arch::SavedFrame, primitive_enum, process::Signal, SyscallFunction};
use kernel_arch::{task::TaskFrame, Architecture};
use kernel_arch::task::TaskFrame;
use libk::{device::external_interrupt_controller, task::thread::Thread};
use tock_registers::interfaces::ReadWriteable;
use kernel_arch_riscv64::{registers::STVEC, ArchitectureImpl};
use kernel_arch_riscv64::registers::STVEC;
use crate::syscall;
use super::timer;
use super::{smp, timer};
primitive_enum! {
pub enum Cause: u64 {
@ -150,7 +150,8 @@ unsafe fn smode_exception_handler(frame: &mut TrapFrame) {
}
_ => (),
}
ArchitectureImpl::halt();
panic!("Unhandled S-mode exception");
}
unsafe extern "C" fn smode_interrupt_handler(frame: *mut TrapFrame) {
@ -158,6 +159,8 @@ unsafe extern "C" fn smode_interrupt_handler(frame: *mut TrapFrame) {
let smode = frame.sstatus & (1 << 8) != 0;
match frame.scause & !(1 << 63) {
// S-mode software interrupt
1 => smp::handle_ipi(),
// S-mode timer interrupt
5 => timer::handle_interrupt(),
// S-mode external interrupt

View File

@ -1,5 +1,5 @@
#![allow(missing_docs)]
use core::sync::atomic::{self, AtomicU32, AtomicUsize, Ordering};
use core::sync::atomic::{self, AtomicU32, Ordering};
use abi::error::Error;
use alloc::sync::Arc;
@ -36,6 +36,7 @@ use super::Platform;
pub mod boot;
pub mod debug;
pub mod exception;
pub mod smp;
pub mod timer;
pub static BOOT_HART_ID: AtomicU32 = AtomicU32::new(u32::MAX);
@ -66,15 +67,15 @@ impl Platform for Riscv64 {
}
unsafe fn send_ipi(&self, target: IpiDeliveryTarget, msg: IpiMessage) -> Result<bool, Error> {
let _ = target;
let _ = msg;
log::warn!("TODO: send_ipi({msg:?})");
Ok(false)
// loop {}
smp::send_ipi(target, msg)?;
Ok(true)
}
unsafe fn start_application_processors(&self) {
// TODO
let dt = self.dt.get();
if let Err(error) = smp::start_secondary_harts(dt) {
log::error!("Couldn't start secondary harts: {error:?}");
}
}
fn register_reset_device(&self, reset: Arc<dyn ResetDevice>) -> Result<(), Error> {
@ -157,10 +158,12 @@ impl Riscv64 {
}
// TODO boot hart ID may not be zero?
unsafe fn init_platform(&'static self, hart_id: u32, is_bsp: bool) -> Result<(), Error> {
static QUEUE_INDEX: AtomicUsize = AtomicUsize::new(0);
let queue_index = QUEUE_INDEX.fetch_add(1, Ordering::AcqRel);
unsafe fn init_platform(
&'static self,
hart_id: u32,
queue_index: usize,
is_bsp: bool,
) -> Result<(), Error> {
let per_cpu = PerCpuData {
tmp_t0: 0,
umode_sp: 0,

View File

@ -0,0 +1,115 @@
use core::{mem::MaybeUninit, sync::atomic::Ordering};
use abi::error::Error;
use device_api::interrupt::{IpiDeliveryTarget, IpiMessage};
use device_tree::{DeviceTree, DeviceTreeNodeExt};
use kernel_arch_riscv64::{mem, registers::SIP, sbi, ArchitectureImpl, CPU_COUNT};
use libk::arch::Cpu;
use libk_mm::{
address::{AsPhysicalAddress, PhysicalAddress},
PageBox,
};
use tock_registers::interfaces::ReadWriteable;
use crate::{arch::riscv64::BOOT_HART_ID, panic};
pub const SECONDARY_STACK_SIZE: usize = 32768;
#[repr(C)]
pub struct SecondaryContext {
// 0x00
pub stack_top: PhysicalAddress,
pub stack: PageBox<[MaybeUninit<u8>]>,
pub hart_id: u64,
}
fn start_secondary_hart(hart_id: u64) -> Result<(), Error> {
extern "C" {
fn __rv64_secondary_entry();
}
let start_addr = __rv64_secondary_entry as usize;
let stack = PageBox::<u8>::new_uninit_slice(SECONDARY_STACK_SIZE)?;
let stack_top = unsafe { PageBox::as_physical_address(&stack).add(SECONDARY_STACK_SIZE) };
let context = PageBox::new(SecondaryContext {
stack,
stack_top,
hart_id,
})?;
let a1 = unsafe { context.as_physical_address() };
log::info!(
"Start secondary hart {hart_id}: pc={:#x}, a1={:#x}",
start_addr,
a1,
);
let old = CPU_COUNT.load(Ordering::Acquire);
sbi::sbi_hart_start(hart_id, start_addr as u64, a1.into_u64())?;
while CPU_COUNT.load(Ordering::Acquire) == old {
core::hint::spin_loop();
}
Ok(())
}
pub fn start_secondary_harts(dt: &DeviceTree) -> Result<(), Error> {
log::info!("Setting up secondary harts");
let boot_hart_id = BOOT_HART_ID.load(Ordering::Acquire);
let cpus = dt.find_absolute("/cpus").ok_or(Error::DoesNotExist)?;
for cpu in cpus.children() {
let Some(reg) = cpu.prop_cell_usize("reg") else {
continue;
};
if reg == boot_hart_id as usize {
continue;
}
if let Err(error) = start_secondary_hart(reg as u64) {
log::error!("hart {reg} start error: {error:?}");
}
}
// Can get rid of lower half now
unsafe { mem::unmap_lower_half() };
Ok(())
}
pub fn send_ipi(target: IpiDeliveryTarget, msg: IpiMessage) -> Result<(), Error> {
let local = Cpu::local();
let mut hart_mask = 0;
match target {
IpiDeliveryTarget::Specific(_) => todo!(),
IpiDeliveryTarget::ThisCpu => todo!(),
IpiDeliveryTarget::OtherCpus => {
ArchitectureImpl::for_each_hart(|hart_id, _, queue| {
if hart_id != local.id() {
hart_mask |= 1 << hart_id;
queue.push(msg);
}
});
}
}
log::debug!("send_ipi({hart_mask:#x}, {msg:?})");
sbi::sbi_send_ipi(hart_mask, 0)
}
pub fn handle_ipi() {
let local = Cpu::local();
if let Some(message) = local.get_ipi() {
match message {
IpiMessage::Panic => panic::panic_secondary(),
IpiMessage::Shutdown => todo!(),
}
} else {
log::warn!("Spurious IPI received");
}
SIP.modify(SIP::SSIP::CLEAR);
}

View File

@ -163,7 +163,10 @@ impl ExternalInterruptController for Plic {
.context
.get();
let mut table = context.table.write();
log::info!("Bind irq #{irq} -> {:?}", handler.display_name());
log::info!(
"Bind irq #{irq} -> hart {bsp_hart_id}, {:?}",
handler.display_name()
);
table.insert(irq as usize, handler)?;