Use CpuId with validity ensurance rather than u32

This commit is contained in:
Zhang Junyang
2024-09-23 11:51:53 +08:00
committed by Tate, Hongliang Tian
parent b400d287fa
commit 3468ec213b
20 changed files with 160 additions and 107 deletions

View File

@ -115,7 +115,7 @@ fn ap_init() -> ! {
let preempt_guard = ostd::task::disable_preempt(); let preempt_guard = ostd::task::disable_preempt();
let cpu_id = preempt_guard.current_cpu(); let cpu_id = preempt_guard.current_cpu();
drop(preempt_guard); drop(preempt_guard);
log::info!("Kernel idle thread for CPU #{} started.", cpu_id); log::info!("Kernel idle thread for CPU #{} started.", cpu_id.as_usize());
loop { loop {
Thread::yield_now(); Thread::yield_now();
} }

View File

@ -164,7 +164,7 @@ static FUTEX_BUCKETS: Once<FutexBucketVec> = Once::new();
/// This number is calculated the same way as Linux's: /// This number is calculated the same way as Linux's:
/// <https://github.com/torvalds/linux/blob/master/kernel/futex/core.c> /// <https://github.com/torvalds/linux/blob/master/kernel/futex/core.c>
fn get_bucket_count() -> usize { fn get_bucket_count() -> usize {
((1 << 8) * num_cpus()).next_power_of_two() as usize ((1 << 8) * num_cpus()).next_power_of_two()
} }
fn get_futex_bucket(key: FutexKey) -> (usize, FutexBucketRef) { fn get_futex_bucket(key: FutexKey) -> (usize, FutexBucketRef) {

View File

@ -1,7 +1,7 @@
// SPDX-License-Identifier: MPL-2.0 // SPDX-License-Identifier: MPL-2.0
use ostd::{ use ostd::{
cpu::{num_cpus, CpuSet, PinCurrentCpu}, cpu::{num_cpus, CpuId, CpuSet, PinCurrentCpu},
sync::PreemptDisabled, sync::PreemptDisabled,
task::{ task::{
scheduler::{ scheduler::{
@ -33,8 +33,8 @@ struct PreemptScheduler<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> {
} }
impl<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> PreemptScheduler<T, U> { impl<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> PreemptScheduler<T, U> {
fn new(nr_cpus: u32) -> Self { fn new(nr_cpus: usize) -> Self {
let mut rq = Vec::with_capacity(nr_cpus as usize); let mut rq = Vec::with_capacity(nr_cpus);
for _ in 0..nr_cpus { for _ in 0..nr_cpus {
rq.push(SpinLock::new(PreemptRunQueue::new())); rq.push(SpinLock::new(PreemptRunQueue::new()));
} }
@ -42,7 +42,7 @@ impl<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> PreemptScheduler<T,
} }
/// Selects a CPU for task to run on for the first time. /// Selects a CPU for task to run on for the first time.
fn select_cpu(&self, entity: &PreemptSchedEntity<T, U>) -> u32 { fn select_cpu(&self, entity: &PreemptSchedEntity<T, U>) -> CpuId {
// If the CPU of a runnable task has been set before, keep scheduling // If the CPU of a runnable task has been set before, keep scheduling
// the task to that one. // the task to that one.
// TODO: Consider migrating tasks between CPUs for load balancing. // TODO: Consider migrating tasks between CPUs for load balancing.
@ -55,7 +55,7 @@ impl<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> PreemptScheduler<T,
let mut minimum_load = usize::MAX; let mut minimum_load = usize::MAX;
for candidate in entity.thread.cpu_affinity().iter() { for candidate in entity.thread.cpu_affinity().iter() {
let rq = self.rq[candidate as usize].lock(); let rq = self.rq[candidate.as_usize()].lock();
// A wild guess measuring the load of a runqueue. We assume that // A wild guess measuring the load of a runqueue. We assume that
// real-time tasks are 4-times as important as normal tasks. // real-time tasks are 4-times as important as normal tasks.
let load = rq.real_time_entities.len() * 8 let load = rq.real_time_entities.len() * 8
@ -74,7 +74,7 @@ impl<T: PreemptSchedInfo + FromTask<U>, U: CommonSchedInfo> PreemptScheduler<T,
impl<T: Sync + Send + PreemptSchedInfo + FromTask<U>, U: Sync + Send + CommonSchedInfo> Scheduler<U> impl<T: Sync + Send + PreemptSchedInfo + FromTask<U>, U: Sync + Send + CommonSchedInfo> Scheduler<U>
for PreemptScheduler<T, U> for PreemptScheduler<T, U>
{ {
fn enqueue(&self, task: Arc<U>, flags: EnqueueFlags) -> Option<u32> { fn enqueue(&self, task: Arc<U>, flags: EnqueueFlags) -> Option<CpuId> {
let entity = PreemptSchedEntity::new(task); let entity = PreemptSchedEntity::new(task);
let mut still_in_rq = false; let mut still_in_rq = false;
let target_cpu = { let target_cpu = {
@ -88,7 +88,7 @@ impl<T: Sync + Send + PreemptSchedInfo + FromTask<U>, U: Sync + Send + CommonSch
cpu_id cpu_id
}; };
let mut rq = self.rq[target_cpu as usize].disable_irq().lock(); let mut rq = self.rq[target_cpu.as_usize()].disable_irq().lock();
if still_in_rq && let Err(_) = entity.task.cpu().set_if_is_none(target_cpu) { if still_in_rq && let Err(_) = entity.task.cpu().set_if_is_none(target_cpu) {
return None; return None;
} }
@ -105,14 +105,14 @@ impl<T: Sync + Send + PreemptSchedInfo + FromTask<U>, U: Sync + Send + CommonSch
fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<U>)) { fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<U>)) {
let irq_guard = disable_local(); let irq_guard = disable_local();
let local_rq: &PreemptRunQueue<T, U> = &self.rq[irq_guard.current_cpu() as usize].lock(); let local_rq: &PreemptRunQueue<T, U> = &self.rq[irq_guard.current_cpu().as_usize()].lock();
f(local_rq); f(local_rq);
} }
fn local_mut_rq_with(&self, f: &mut dyn FnMut(&mut dyn LocalRunQueue<U>)) { fn local_mut_rq_with(&self, f: &mut dyn FnMut(&mut dyn LocalRunQueue<U>)) {
let irq_guard = disable_local(); let irq_guard = disable_local();
let local_rq: &mut PreemptRunQueue<T, U> = let local_rq: &mut PreemptRunQueue<T, U> =
&mut self.rq[irq_guard.current_cpu() as usize].lock(); &mut self.rq[irq_guard.current_cpu().as_usize()].lock();
f(local_rq); f(local_rq);
} }
} }

View File

@ -64,7 +64,7 @@
//! //!
//! ``` //! ```
use ostd::cpu::CpuSet; use ostd::cpu::{CpuId, CpuSet};
use spin::Once; use spin::Once;
use work_item::WorkItem; use work_item::WorkItem;
use worker_pool::WorkerPool; use worker_pool::WorkerPool;
@ -150,7 +150,7 @@ impl WorkQueue {
/// Request a pending work item. The `request_cpu` indicates the CPU where /// Request a pending work item. The `request_cpu` indicates the CPU where
/// the calling worker is located. /// the calling worker is located.
fn dequeue(&self, request_cpu: u32) -> Option<Arc<WorkItem>> { fn dequeue(&self, request_cpu: CpuId) -> Option<Arc<WorkItem>> {
let mut inner = self.inner.disable_irq().lock(); let mut inner = self.inner.disable_irq().lock();
let index = inner let index = inner
.pending_work_items .pending_work_items
@ -160,7 +160,7 @@ impl WorkQueue {
Some(item) Some(item)
} }
fn has_pending_work_items(&self, request_cpu: u32) -> bool { fn has_pending_work_items(&self, request_cpu: CpuId) -> bool {
self.inner self.inner
.disable_irq() .disable_irq()
.lock() .lock()

View File

@ -4,7 +4,7 @@
use core::sync::atomic::{AtomicBool, Ordering}; use core::sync::atomic::{AtomicBool, Ordering};
use ostd::cpu::CpuSet; use ostd::cpu::{CpuId, CpuSet};
use crate::prelude::*; use crate::prelude::*;
@ -33,7 +33,7 @@ impl WorkItem {
&mut self.cpu_affinity &mut self.cpu_affinity
} }
pub(super) fn is_valid_cpu(&self, cpu_id: u32) -> bool { pub(super) fn is_valid_cpu(&self, cpu_id: CpuId) -> bool {
self.cpu_affinity.contains(cpu_id) self.cpu_affinity.contains(cpu_id)
} }

View File

@ -2,7 +2,10 @@
#![allow(dead_code)] #![allow(dead_code)]
use ostd::{cpu::CpuSet, task::Task}; use ostd::{
cpu::{CpuId, CpuSet},
task::Task,
};
use super::worker_pool::WorkerPool; use super::worker_pool::WorkerPool;
use crate::{ use crate::{
@ -19,7 +22,7 @@ use crate::{
pub(super) struct Worker { pub(super) struct Worker {
worker_pool: Weak<WorkerPool>, worker_pool: Weak<WorkerPool>,
bound_task: Arc<Task>, bound_task: Arc<Task>,
bound_cpu: u32, bound_cpu: CpuId,
inner: SpinLock<WorkerInner>, inner: SpinLock<WorkerInner>,
} }
@ -39,7 +42,7 @@ enum WorkerStatus {
impl Worker { impl Worker {
/// Creates a new `Worker` to the given `worker_pool`. /// Creates a new `Worker` to the given `worker_pool`.
pub(super) fn new(worker_pool: Weak<WorkerPool>, bound_cpu: u32) -> Arc<Self> { pub(super) fn new(worker_pool: Weak<WorkerPool>, bound_cpu: CpuId) -> Arc<Self> {
Arc::new_cyclic(|worker_ref| { Arc::new_cyclic(|worker_ref| {
let weal_worker = worker_ref.clone(); let weal_worker = worker_ref.clone();
let task_fn = Box::new(move || { let task_fn = Box::new(move || {

View File

@ -7,7 +7,11 @@ use core::{
time::Duration, time::Duration,
}; };
use ostd::{cpu::CpuSet, sync::WaitQueue, task::Task}; use ostd::{
cpu::{CpuId, CpuSet},
sync::WaitQueue,
task::Task,
};
use super::{simple_scheduler::SimpleScheduler, worker::Worker, WorkItem, WorkPriority, WorkQueue}; use super::{simple_scheduler::SimpleScheduler, worker::Worker, WorkItem, WorkPriority, WorkQueue};
use crate::{ use crate::{
@ -34,7 +38,7 @@ pub struct WorkerPool {
/// A set of workers for a specific CPU. /// A set of workers for a specific CPU.
pub struct LocalWorkerPool { pub struct LocalWorkerPool {
cpu_id: u32, cpu_id: CpuId,
idle_wait_queue: WaitQueue, idle_wait_queue: WaitQueue,
parent: Weak<WorkerPool>, parent: Weak<WorkerPool>,
/// A liveness check for LocalWorkerPool. The monitor periodically clears heartbeat, /// A liveness check for LocalWorkerPool. The monitor periodically clears heartbeat,
@ -66,7 +70,7 @@ pub struct Monitor {
} }
impl LocalWorkerPool { impl LocalWorkerPool {
fn new(worker_pool: Weak<WorkerPool>, cpu_id: u32) -> Self { fn new(worker_pool: Weak<WorkerPool>, cpu_id: CpuId) -> Self {
LocalWorkerPool { LocalWorkerPool {
cpu_id, cpu_id,
idle_wait_queue: WaitQueue::new(), idle_wait_queue: WaitQueue::new(),
@ -151,7 +155,7 @@ impl WorkerPool {
self.work_queues.disable_irq().lock().push(work_queue); self.work_queues.disable_irq().lock().push(work_queue);
} }
pub fn has_pending_work_items(&self, request_cpu: u32) -> bool { pub fn has_pending_work_items(&self, request_cpu: CpuId) -> bool {
self.work_queues self.work_queues
.disable_irq() .disable_irq()
.lock() .lock()
@ -163,7 +167,7 @@ impl WorkerPool {
self.scheduler.schedule(); self.scheduler.schedule();
} }
pub fn num_workers(&self, cpu_id: u32) -> u16 { pub fn num_workers(&self, cpu_id: CpuId) -> u16 {
self.local_pool(cpu_id).workers.disable_irq().lock().len() as u16 self.local_pool(cpu_id).workers.disable_irq().lock().len() as u16
} }
@ -171,7 +175,7 @@ impl WorkerPool {
&self.cpu_set &self.cpu_set
} }
pub(super) fn fetch_pending_work_item(&self, request_cpu: u32) -> Option<Arc<WorkItem>> { pub(super) fn fetch_pending_work_item(&self, request_cpu: CpuId) -> Option<Arc<WorkItem>> {
for work_queue in self.work_queues.disable_irq().lock().iter() { for work_queue in self.work_queues.disable_irq().lock().iter() {
let item = work_queue.dequeue(request_cpu); let item = work_queue.dequeue(request_cpu);
if item.is_some() { if item.is_some() {
@ -181,22 +185,22 @@ impl WorkerPool {
None None
} }
fn local_pool(&self, cpu_id: u32) -> &Arc<LocalWorkerPool> { fn local_pool(&self, cpu_id: CpuId) -> &Arc<LocalWorkerPool> {
self.local_pools self.local_pools
.iter() .iter()
.find(|local_pool: &&Arc<LocalWorkerPool>| local_pool.cpu_id == cpu_id) .find(|local_pool: &&Arc<LocalWorkerPool>| local_pool.cpu_id == cpu_id)
.unwrap() .unwrap()
} }
pub(super) fn wake_worker(&self, cpu_id: u32) -> bool { pub(super) fn wake_worker(&self, cpu_id: CpuId) -> bool {
self.local_pool(cpu_id).wake_worker() self.local_pool(cpu_id).wake_worker()
} }
pub(super) fn add_worker(&self, cpu_id: u32) { pub(super) fn add_worker(&self, cpu_id: CpuId) {
self.local_pool(cpu_id).add_worker(); self.local_pool(cpu_id).add_worker();
} }
pub(super) fn remove_worker(&self, cpu_id: u32) { pub(super) fn remove_worker(&self, cpu_id: CpuId) {
self.local_pool(cpu_id).remove_worker(); self.local_pool(cpu_id).remove_worker();
} }
@ -204,15 +208,15 @@ impl WorkerPool {
self.priority == WorkPriority::High self.priority == WorkPriority::High
} }
pub(super) fn heartbeat(&self, cpu_id: u32) -> bool { pub(super) fn heartbeat(&self, cpu_id: CpuId) -> bool {
self.local_pool(cpu_id).heartbeat() self.local_pool(cpu_id).heartbeat()
} }
pub(super) fn set_heartbeat(&self, cpu_id: u32, heartbeat: bool) { pub(super) fn set_heartbeat(&self, cpu_id: CpuId, heartbeat: bool) {
self.local_pool(cpu_id).set_heartbeat(heartbeat) self.local_pool(cpu_id).set_heartbeat(heartbeat)
} }
pub(super) fn idle_current_worker(&self, cpu_id: u32, worker: Arc<Worker>) { pub(super) fn idle_current_worker(&self, cpu_id: CpuId, worker: Arc<Worker>) {
self.local_pool(cpu_id).idle_current_worker(worker); self.local_pool(cpu_id).idle_current_worker(worker);
} }
} }

View File

@ -4,13 +4,7 @@ use alloc::sync::Arc;
use core::time::Duration; use core::time::Duration;
use aster_time::read_monotonic_time; use aster_time::read_monotonic_time;
use ostd::{ use ostd::{cpu::PinCurrentCpu, cpu_local, sync::SpinLock, task::disable_preempt, timer::Jiffies};
cpu::{num_cpus, PinCurrentCpu},
cpu_local,
sync::SpinLock,
task::disable_preempt,
timer::Jiffies,
};
use paste::paste; use paste::paste;
use spin::Once; use spin::Once;
@ -232,7 +226,7 @@ macro_rules! define_timer_managers {
$( $(
let clock = paste! {[<$clock_id _INSTANCE>].get().unwrap().clone()}; let clock = paste! {[<$clock_id _INSTANCE>].get().unwrap().clone()};
let clock_manager = TimerManager::new(clock); let clock_manager = TimerManager::new(clock);
for cpu in 0..num_cpus() { for cpu in ostd::cpu::all_cpus() {
paste! { paste! {
[<$clock_id _MANAGER>].get_on_cpu(cpu).call_once(|| clock_manager.clone()); [<$clock_id _MANAGER>].get_on_cpu(cpu).call_once(|| clock_manager.clone());
} }
@ -307,7 +301,7 @@ pub(super) fn init() {
/// to avoid functions like this one. /// to avoid functions like this one.
pub fn init_for_ktest() { pub fn init_for_ktest() {
// If `spin::Once` has initialized, this closure will not be executed. // If `spin::Once` has initialized, this closure will not be executed.
for cpu in 0..num_cpus() { for cpu in ostd::cpu::all_cpus() {
CLOCK_REALTIME_MANAGER.get_on_cpu(cpu).call_once(|| { CLOCK_REALTIME_MANAGER.get_on_cpu(cpu).call_once(|| {
let clock = RealTimeClock { _private: () }; let clock = RealTimeClock { _private: () };
TimerManager::new(Arc::new(clock)) TimerManager::new(Arc::new(clock))

View File

@ -8,6 +8,7 @@ use id_alloc::IdAlloc;
use spin::Once; use spin::Once;
use crate::{ use crate::{
cpu::CpuId,
sync::{Mutex, PreemptDisabled, SpinLock, SpinLockGuard}, sync::{Mutex, PreemptDisabled, SpinLock, SpinLockGuard},
trap::TrapFrame, trap::TrapFrame,
}; };
@ -145,6 +146,6 @@ impl Drop for IrqCallbackHandle {
/// ///
/// The caller must ensure that the CPU ID and the interrupt number corresponds /// The caller must ensure that the CPU ID and the interrupt number corresponds
/// to a safe function to call. /// to a safe function to call.
pub(crate) unsafe fn send_ipi(cpu_id: u32, irq_num: u8) { pub(crate) unsafe fn send_ipi(cpu_id: CpuId, irq_num: u8) {
unimplemented!() unimplemented!()
} }

View File

@ -11,6 +11,7 @@ use spin::Once;
use x86_64::registers::rflags::{self, RFlags}; use x86_64::registers::rflags::{self, RFlags};
use crate::{ use crate::{
cpu::CpuId,
sync::{Mutex, RwLock, RwLockReadGuard, SpinLock}, sync::{Mutex, RwLock, RwLockReadGuard, SpinLock},
trap::TrapFrame, trap::TrapFrame,
}; };
@ -160,11 +161,11 @@ impl Drop for IrqCallbackHandle {
/// ///
/// The caller must ensure that the CPU ID and the interrupt number corresponds /// The caller must ensure that the CPU ID and the interrupt number corresponds
/// to a safe function to call. /// to a safe function to call.
pub(crate) unsafe fn send_ipi(cpu_id: u32, irq_num: u8) { pub(crate) unsafe fn send_ipi(cpu_id: CpuId, irq_num: u8) {
use crate::arch::kernel::apic::{self, Icr}; use crate::arch::kernel::apic::{self, Icr};
let icr = Icr::new( let icr = Icr::new(
apic::ApicId::from(cpu_id), apic::ApicId::from(cpu_id.as_usize() as u32),
apic::DestinationShorthand::NoShorthand, apic::DestinationShorthand::NoShorthand,
apic::TriggerMode::Edge, apic::TriggerMode::Edge,
apic::Level::Assert, apic::Level::Assert,

View File

@ -5,7 +5,7 @@
use core::{marker::Sync, ops::Deref}; use core::{marker::Sync, ops::Deref};
use super::{__cpu_local_end, __cpu_local_start}; use super::{__cpu_local_end, __cpu_local_start};
use crate::{arch, trap::DisabledLocalIrqGuard}; use crate::{arch, cpu::CpuId, trap::DisabledLocalIrqGuard};
/// Defines a CPU-local variable. /// Defines a CPU-local variable.
/// ///
@ -139,9 +139,11 @@ impl<T: 'static + Sync> CpuLocal<T> {
/// # Panics /// # Panics
/// ///
/// Panics if the CPU ID is out of range. /// Panics if the CPU ID is out of range.
pub fn get_on_cpu(&'static self, cpu_id: u32) -> &'static T { pub fn get_on_cpu(&'static self, cpu_id: CpuId) -> &'static T {
super::has_init::assert_true(); super::has_init::assert_true();
let cpu_id = cpu_id.as_usize();
// If on the BSP, just use the statically linked storage. // If on the BSP, just use the statically linked storage.
if cpu_id == 0 { if cpu_id == 0 {
return &self.0; return &self.0;
@ -153,7 +155,7 @@ impl<T: 'static + Sync> CpuLocal<T> {
let base = unsafe { let base = unsafe {
super::CPU_LOCAL_STORAGES super::CPU_LOCAL_STORAGES
.get_unchecked() .get_unchecked()
.get(cpu_id as usize - 1) .get(cpu_id - 1)
.unwrap() .unwrap()
.start_paddr() .start_paddr()
}; };

View File

@ -95,7 +95,7 @@ pub unsafe fn init_on_bsp() {
let num_cpus = super::num_cpus(); let num_cpus = super::num_cpus();
let mut cpu_local_storages = Vec::with_capacity(num_cpus as usize - 1); let mut cpu_local_storages = Vec::with_capacity(num_cpus - 1);
for _ in 1..num_cpus { for _ in 1..num_cpus {
let ap_pages = { let ap_pages = {
let nbytes = (bsp_end_va - bsp_base_va).align_up(PAGE_SIZE); let nbytes = (bsp_end_va - bsp_base_va).align_up(PAGE_SIZE);

View File

@ -21,6 +21,37 @@ use crate::{
trap::DisabledLocalIrqGuard, trap::DisabledLocalIrqGuard,
}; };
/// The ID of a CPU in the system.
///
/// If converting from/to an integer, the integer must start from 0 and be less
/// than the number of CPUs.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct CpuId(u32);
impl CpuId {
/// Returns the CPU ID of the bootstrap processor (BSP).
pub const fn bsp() -> Self {
CpuId(0)
}
/// Converts the CPU ID to an `usize`.
pub const fn as_usize(self) -> usize {
self.0 as usize
}
}
impl TryFrom<usize> for CpuId {
type Error = &'static str;
fn try_from(value: usize) -> Result<Self, Self::Error> {
if value < num_cpus() {
Ok(CpuId(value as u32))
} else {
Err("The given CPU ID is out of range")
}
}
}
/// The number of CPUs. /// The number of CPUs.
static NUM_CPUS: Once<u32> = Once::new(); static NUM_CPUS: Once<u32> = Once::new();
@ -46,14 +77,20 @@ pub(crate) unsafe fn set_this_cpu_id(id: u32) {
} }
/// Returns the number of CPUs. /// Returns the number of CPUs.
pub fn num_cpus() -> u32 { pub fn num_cpus() -> usize {
debug_assert!( debug_assert!(
NUM_CPUS.get().is_some(), NUM_CPUS.get().is_some(),
"The number of CPUs is not initialized" "The number of CPUs is not initialized"
); );
// SAFETY: The number of CPUs is initialized. The unsafe version is used // SAFETY: The number of CPUs is initialized. The unsafe version is used
// to avoid the overhead of the check. // to avoid the overhead of the check.
unsafe { *NUM_CPUS.get_unchecked() } let num = unsafe { *NUM_CPUS.get_unchecked() };
num as usize
}
/// Returns an iterator over all CPUs.
pub fn all_cpus() -> impl Iterator<Item = CpuId> {
(0..num_cpus()).map(|id| CpuId(id as u32))
} }
/// A marker trait for guard types that can "pin" the current task to the /// A marker trait for guard types that can "pin" the current task to the
@ -70,10 +107,10 @@ pub fn num_cpus() -> u32 {
/// CPU while any one of the instances of the implemented structure exists. /// CPU while any one of the instances of the implemented structure exists.
pub unsafe trait PinCurrentCpu { pub unsafe trait PinCurrentCpu {
/// Returns the number of the current CPU. /// Returns the number of the current CPU.
fn current_cpu(&self) -> u32 { fn current_cpu(&self) -> CpuId {
let id = CURRENT_CPU.load(); let id = CURRENT_CPU.load();
debug_assert_ne!(id, u32::MAX, "This CPU is not initialized"); debug_assert_ne!(id, u32::MAX, "This CPU is not initialized");
id CpuId(id)
} }
} }

View File

@ -7,7 +7,7 @@ use core::sync::atomic::{AtomicU64, Ordering};
use smallvec::SmallVec; use smallvec::SmallVec;
use static_assertions::const_assert_eq; use static_assertions::const_assert_eq;
use super::num_cpus; use super::{num_cpus, CpuId};
/// A subset of all CPUs in the system. /// A subset of all CPUs in the system.
#[derive(Clone, Debug, Default)] #[derive(Clone, Debug, Default)]
@ -21,12 +21,12 @@ type InnerPart = u64;
const BITS_PER_PART: usize = core::mem::size_of::<InnerPart>() * 8; const BITS_PER_PART: usize = core::mem::size_of::<InnerPart>() * 8;
const NR_PARTS_NO_ALLOC: usize = 2; const NR_PARTS_NO_ALLOC: usize = 2;
const fn part_idx(cpu_id: u32) -> usize { const fn part_idx(cpu_id: CpuId) -> usize {
cpu_id as usize / BITS_PER_PART cpu_id.as_usize() / BITS_PER_PART
} }
const fn bit_idx(cpu_id: u32) -> usize { const fn bit_idx(cpu_id: CpuId) -> usize {
cpu_id as usize % BITS_PER_PART cpu_id.as_usize() % BITS_PER_PART
} }
const fn parts_for_cpus(num_cpus: usize) -> usize { const fn parts_for_cpus(num_cpus: usize) -> usize {
@ -36,18 +36,18 @@ const fn parts_for_cpus(num_cpus: usize) -> usize {
impl CpuSet { impl CpuSet {
/// Creates a new `CpuSet` with all CPUs in the system. /// Creates a new `CpuSet` with all CPUs in the system.
pub fn new_full() -> Self { pub fn new_full() -> Self {
let mut ret = Self::with_capacity_val(num_cpus() as usize, !0); let mut ret = Self::with_capacity_val(num_cpus(), !0);
ret.clear_nonexistent_cpu_bits(); ret.clear_nonexistent_cpu_bits();
ret ret
} }
/// Creates a new `CpuSet` with no CPUs in the system. /// Creates a new `CpuSet` with no CPUs in the system.
pub fn new_empty() -> Self { pub fn new_empty() -> Self {
Self::with_capacity_val(num_cpus() as usize, 0) Self::with_capacity_val(num_cpus(), 0)
} }
/// Adds a CPU to the set. /// Adds a CPU to the set.
pub fn add(&mut self, cpu_id: u32) { pub fn add(&mut self, cpu_id: CpuId) {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
if part_idx >= self.bits.len() { if part_idx >= self.bits.len() {
@ -57,7 +57,7 @@ impl CpuSet {
} }
/// Removes a CPU from the set. /// Removes a CPU from the set.
pub fn remove(&mut self, cpu_id: u32) { pub fn remove(&mut self, cpu_id: CpuId) {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
if part_idx < self.bits.len() { if part_idx < self.bits.len() {
@ -66,7 +66,7 @@ impl CpuSet {
} }
/// Returns true if the set contains the specified CPU. /// Returns true if the set contains the specified CPU.
pub fn contains(&self, cpu_id: u32) -> bool { pub fn contains(&self, cpu_id: CpuId) -> bool {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
part_idx < self.bits.len() && (self.bits[part_idx] & (1 << bit_idx)) != 0 part_idx < self.bits.len() && (self.bits[part_idx] & (1 << bit_idx)) != 0
@ -92,11 +92,12 @@ impl CpuSet {
} }
/// Iterates over the CPUs in the set. /// Iterates over the CPUs in the set.
pub fn iter(&self) -> impl Iterator<Item = u32> + '_ { pub fn iter(&self) -> impl Iterator<Item = CpuId> + '_ {
self.bits.iter().enumerate().flat_map(|(part_idx, &part)| { self.bits.iter().enumerate().flat_map(|(part_idx, &part)| {
(0..BITS_PER_PART).filter_map(move |bit_idx| { (0..BITS_PER_PART).filter_map(move |bit_idx| {
if (part & (1 << bit_idx)) != 0 { if (part & (1 << bit_idx)) != 0 {
Some((part_idx * BITS_PER_PART + bit_idx) as u32) let id = part_idx * BITS_PER_PART + bit_idx;
Some(CpuId(id as u32))
} else { } else {
None None
} }
@ -113,7 +114,7 @@ impl CpuSet {
} }
fn clear_nonexistent_cpu_bits(&mut self) { fn clear_nonexistent_cpu_bits(&mut self) {
let num_cpus = num_cpus() as usize; let num_cpus = num_cpus();
if num_cpus % BITS_PER_PART != 0 { if num_cpus % BITS_PER_PART != 0 {
let num_parts = parts_for_cpus(num_cpus); let num_parts = parts_for_cpus(num_cpus);
self.bits[num_parts - 1] &= (1 << (num_cpus % BITS_PER_PART)) - 1; self.bits[num_parts - 1] &= (1 << (num_cpus % BITS_PER_PART)) - 1;
@ -121,8 +122,8 @@ impl CpuSet {
} }
} }
impl From<u32> for CpuSet { impl From<CpuId> for CpuSet {
fn from(cpu_id: u32) -> Self { fn from(cpu_id: CpuId) -> Self {
let mut set = Self::new_empty(); let mut set = Self::new_empty();
set.add(cpu_id); set.add(cpu_id);
set set
@ -171,7 +172,7 @@ impl AtomicCpuSet {
} }
/// Atomically adds a CPU with the given ordering. /// Atomically adds a CPU with the given ordering.
pub fn add(&self, cpu_id: u32, ordering: Ordering) { pub fn add(&self, cpu_id: CpuId, ordering: Ordering) {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
if part_idx < self.bits.len() { if part_idx < self.bits.len() {
@ -180,7 +181,7 @@ impl AtomicCpuSet {
} }
/// Atomically removes a CPU with the given ordering. /// Atomically removes a CPU with the given ordering.
pub fn remove(&self, cpu_id: u32, ordering: Ordering) { pub fn remove(&self, cpu_id: CpuId, ordering: Ordering) {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
if part_idx < self.bits.len() { if part_idx < self.bits.len() {
@ -189,7 +190,7 @@ impl AtomicCpuSet {
} }
/// Atomically checks if the set contains the specified CPU. /// Atomically checks if the set contains the specified CPU.
pub fn contains(&self, cpu_id: u32, ordering: Ordering) -> bool { pub fn contains(&self, cpu_id: CpuId, ordering: Ordering) -> bool {
let part_idx = part_idx(cpu_id); let part_idx = part_idx(cpu_id);
let bit_idx = bit_idx(cpu_id); let bit_idx = bit_idx(cpu_id);
part_idx < self.bits.len() && (self.bits[part_idx].load(ordering) & (1 << bit_idx)) != 0 part_idx < self.bits.len() && (self.bits[part_idx].load(ordering) & (1 << bit_idx)) != 0
@ -199,24 +200,23 @@ impl AtomicCpuSet {
#[cfg(ktest)] #[cfg(ktest)]
mod test { mod test {
use super::*; use super::*;
use crate::prelude::*; use crate::{cpu::all_cpus, prelude::*};
#[ktest] #[ktest]
fn test_full_cpu_set_iter_is_all() { fn test_full_cpu_set_iter_is_all() {
let set = CpuSet::new_full(); let set = CpuSet::new_full();
let num_cpus = num_cpus(); let num_cpus = num_cpus();
let all_cpus = (0..num_cpus).collect::<Vec<_>>(); let all_cpus = all_cpus().collect::<Vec<_>>();
let set_cpus = set.iter().collect::<Vec<_>>(); let set_cpus = set.iter().collect::<Vec<_>>();
assert!(set_cpus.len() == num_cpus as usize); assert!(set_cpus.len() == num_cpus);
assert_eq!(set_cpus, all_cpus); assert_eq!(set_cpus, all_cpus);
} }
#[ktest] #[ktest]
fn test_full_cpu_set_contains_all() { fn test_full_cpu_set_contains_all() {
let set = CpuSet::new_full(); let set = CpuSet::new_full();
let num_cpus = num_cpus(); for cpu_id in all_cpus() {
for cpu_id in 0..num_cpus {
assert!(set.contains(cpu_id)); assert!(set.contains(cpu_id));
} }
} }
@ -231,28 +231,29 @@ mod test {
#[ktest] #[ktest]
fn test_empty_cpu_set_contains_none() { fn test_empty_cpu_set_contains_none() {
let set = CpuSet::new_empty(); let set = CpuSet::new_empty();
let num_cpus = num_cpus(); for cpu_id in all_cpus() {
for cpu_id in 0..num_cpus {
assert!(!set.contains(cpu_id)); assert!(!set.contains(cpu_id));
} }
} }
#[ktest] #[ktest]
fn test_atomic_cpu_set_multiple_sizes() { fn test_atomic_cpu_set_multiple_sizes() {
for test_num_cpus in [1, 3, 12, 64, 96, 99, 128, 256, 288, 1024] { for test_num_cpus in [1usize, 3, 12, 64, 96, 99, 128, 256, 288, 1024] {
let test_all_iter = || (0..test_num_cpus).map(|id| CpuId(id as u32));
let set = CpuSet::with_capacity_val(test_num_cpus, 0); let set = CpuSet::with_capacity_val(test_num_cpus, 0);
let atomic_set = AtomicCpuSet::new(set); let atomic_set = AtomicCpuSet::new(set);
for cpu_id in 0..test_num_cpus as u32 { for cpu_id in test_all_iter() {
assert!(!atomic_set.contains(cpu_id, Ordering::Relaxed)); assert!(!atomic_set.contains(cpu_id, Ordering::Relaxed));
if cpu_id % 3 == 0 { if cpu_id.as_usize() % 3 == 0 {
atomic_set.add(cpu_id, Ordering::Relaxed); atomic_set.add(cpu_id, Ordering::Relaxed);
} }
} }
let loaded = atomic_set.load(); let loaded = atomic_set.load();
for cpu_id in loaded.iter() { for cpu_id in loaded.iter() {
if cpu_id % 3 == 0 { if cpu_id.as_usize() % 3 == 0 {
assert!(loaded.contains(cpu_id)); assert!(loaded.contains(cpu_id));
} else { } else {
assert!(!loaded.contains(cpu_id)); assert!(!loaded.contains(cpu_id));
@ -261,7 +262,7 @@ mod test {
atomic_set.store(CpuSet::with_capacity_val(test_num_cpus, 0)); atomic_set.store(CpuSet::with_capacity_val(test_num_cpus, 0));
for cpu_id in 0..test_num_cpus as u32 { for cpu_id in test_all_iter() {
assert!(!atomic_set.contains(cpu_id, Ordering::Relaxed)); assert!(!atomic_set.contains(cpu_id, Ordering::Relaxed));
atomic_set.add(cpu_id, Ordering::Relaxed); atomic_set.add(cpu_id, Ordering::Relaxed);
} }

View File

@ -68,7 +68,7 @@ where
/// another page table and before this dismissal. /// another page table and before this dismissal.
pub(crate) unsafe fn dismiss() { pub(crate) unsafe fn dismiss() {
IS_DISMISSED.store(true); IS_DISMISSED.store(true);
if DISMISS_COUNT.fetch_add(1, Ordering::SeqCst) == num_cpus() - 1 { if DISMISS_COUNT.fetch_add(1, Ordering::SeqCst) as usize == num_cpus() - 1 {
BOOT_PAGE_TABLE.lock().take(); BOOT_PAGE_TABLE.lock().take();
} }
} }

View File

@ -16,7 +16,7 @@ use core::{
use crate::{ use crate::{
arch::mm::{current_page_table_paddr, PageTableEntry, PagingConsts}, arch::mm::{current_page_table_paddr, PageTableEntry, PagingConsts},
cpu::{num_cpus, CpuExceptionInfo, CpuSet, PinCurrentCpu}, cpu::{all_cpus, CpuExceptionInfo, CpuSet, PinCurrentCpu},
cpu_local, cpu_local,
mm::{ mm::{
io::Fallible, io::Fallible,
@ -93,7 +93,7 @@ impl VmSpace {
let mut activated_cpus = CpuSet::new_empty(); let mut activated_cpus = CpuSet::new_empty();
for cpu in 0..num_cpus() { for cpu in all_cpus() {
// The activation lock is held; other CPUs cannot activate this `VmSpace`. // The activation lock is held; other CPUs cannot activate this `VmSpace`.
let ptr = let ptr =
ACTIVATED_VM_SPACE.get_on_cpu(cpu).load(Ordering::Relaxed) as *const VmSpace; ACTIVATED_VM_SPACE.get_on_cpu(cpu).load(Ordering::Relaxed) as *const VmSpace;

View File

@ -72,7 +72,7 @@ fn do_inter_processor_call(_trapframe: &TrapFrame) {
let mut queue = CALL_QUEUES.get_on_cpu(cur_cpu).lock(); let mut queue = CALL_QUEUES.get_on_cpu(cur_cpu).lock();
while let Some(f) = queue.pop_front() { while let Some(f) = queue.pop_front() {
log::trace!( log::trace!(
"Performing inter-processor call to {:#?} on CPU {}", "Performing inter-processor call to {:#?} on CPU {:#?}",
f, f,
cur_cpu cur_cpu
); );

View File

@ -6,7 +6,7 @@ use super::{
info::CommonSchedInfo, inject_scheduler, EnqueueFlags, LocalRunQueue, Scheduler, UpdateFlags, info::CommonSchedInfo, inject_scheduler, EnqueueFlags, LocalRunQueue, Scheduler, UpdateFlags,
}; };
use crate::{ use crate::{
cpu::{num_cpus, PinCurrentCpu}, cpu::{num_cpus, CpuId, PinCurrentCpu},
sync::SpinLock, sync::SpinLock,
task::{disable_preempt, Task}, task::{disable_preempt, Task},
}; };
@ -25,7 +25,7 @@ struct FifoScheduler<T: CommonSchedInfo> {
impl<T: CommonSchedInfo> FifoScheduler<T> { impl<T: CommonSchedInfo> FifoScheduler<T> {
/// Creates a new instance of `FifoScheduler`. /// Creates a new instance of `FifoScheduler`.
fn new(nr_cpus: u32) -> Self { fn new(nr_cpus: usize) -> Self {
let mut rq = Vec::new(); let mut rq = Vec::new();
for _ in 0..nr_cpus { for _ in 0..nr_cpus {
rq.push(SpinLock::new(FifoRunQueue::new())); rq.push(SpinLock::new(FifoRunQueue::new()));
@ -33,14 +33,14 @@ impl<T: CommonSchedInfo> FifoScheduler<T> {
Self { rq } Self { rq }
} }
fn select_cpu(&self) -> u32 { fn select_cpu(&self) -> CpuId {
// FIXME: adopt more reasonable policy once we fully enable SMP. // FIXME: adopt more reasonable policy once we fully enable SMP.
0 CpuId::bsp()
} }
} }
impl<T: CommonSchedInfo + Send + Sync> Scheduler<T> for FifoScheduler<T> { impl<T: CommonSchedInfo + Send + Sync> Scheduler<T> for FifoScheduler<T> {
fn enqueue(&self, runnable: Arc<T>, flags: EnqueueFlags) -> Option<u32> { fn enqueue(&self, runnable: Arc<T>, flags: EnqueueFlags) -> Option<CpuId> {
let mut still_in_rq = false; let mut still_in_rq = false;
let target_cpu = { let target_cpu = {
let mut cpu_id = self.select_cpu(); let mut cpu_id = self.select_cpu();
@ -53,7 +53,7 @@ impl<T: CommonSchedInfo + Send + Sync> Scheduler<T> for FifoScheduler<T> {
cpu_id cpu_id
}; };
let mut rq = self.rq[target_cpu as usize].disable_irq().lock(); let mut rq = self.rq[target_cpu.as_usize()].disable_irq().lock();
if still_in_rq && let Err(_) = runnable.cpu().set_if_is_none(target_cpu) { if still_in_rq && let Err(_) = runnable.cpu().set_if_is_none(target_cpu) {
return None; return None;
} }
@ -64,7 +64,7 @@ impl<T: CommonSchedInfo + Send + Sync> Scheduler<T> for FifoScheduler<T> {
fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<T>)) { fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<T>)) {
let preempt_guard = disable_preempt(); let preempt_guard = disable_preempt();
let local_rq: &FifoRunQueue<T> = &self.rq[preempt_guard.current_cpu() as usize] let local_rq: &FifoRunQueue<T> = &self.rq[preempt_guard.current_cpu().as_usize()]
.disable_irq() .disable_irq()
.lock(); .lock();
f(local_rq); f(local_rq);
@ -72,7 +72,7 @@ impl<T: CommonSchedInfo + Send + Sync> Scheduler<T> for FifoScheduler<T> {
fn local_mut_rq_with(&self, f: &mut dyn FnMut(&mut dyn LocalRunQueue<T>)) { fn local_mut_rq_with(&self, f: &mut dyn FnMut(&mut dyn LocalRunQueue<T>)) {
let preempt_guard = disable_preempt(); let preempt_guard = disable_preempt();
let local_rq: &mut FifoRunQueue<T> = &mut self.rq[preempt_guard.current_cpu() as usize] let local_rq: &mut FifoRunQueue<T> = &mut self.rq[preempt_guard.current_cpu().as_usize()]
.disable_irq() .disable_irq()
.lock(); .lock();
f(local_rq); f(local_rq);

View File

@ -4,7 +4,7 @@
use core::sync::atomic::{AtomicU32, Ordering}; use core::sync::atomic::{AtomicU32, Ordering};
use crate::task::Task; use crate::{cpu::CpuId, task::Task};
/// Fields of a task that OSTD will never touch. /// Fields of a task that OSTD will never touch.
/// ///
@ -28,17 +28,22 @@ impl AtomicCpuId {
/// An `AtomicCpuId` with `AtomicCpuId::NONE` as its inner value is empty. /// An `AtomicCpuId` with `AtomicCpuId::NONE` as its inner value is empty.
const NONE: u32 = u32::MAX; const NONE: u32 = u32::MAX;
fn new(cpu_id: u32) -> Self {
Self(AtomicU32::new(cpu_id))
}
/// Sets the inner value of an `AtomicCpuId` if it's empty. /// Sets the inner value of an `AtomicCpuId` if it's empty.
/// ///
/// The return value is a result indicating whether the new value was written /// The return value is a result indicating whether the new value was written
/// and containing the previous value. /// and containing the previous value. If the previous value is empty, it returns
pub fn set_if_is_none(&self, cpu_id: u32) -> core::result::Result<u32, u32> { /// `Ok(())`. Otherwise, it returns `Err(previous_value)` which the previous
/// value is a valid CPU ID.
pub fn set_if_is_none(&self, cpu_id: CpuId) -> core::result::Result<(), CpuId> {
self.0 self.0
.compare_exchange(Self::NONE, cpu_id, Ordering::Relaxed, Ordering::Relaxed) .compare_exchange(
Self::NONE,
cpu_id.as_usize() as u32,
Ordering::Relaxed,
Ordering::Relaxed,
)
.map(|_| ())
.map_err(|prev| (prev as usize).try_into().unwrap())
} }
/// Sets the inner value of an `AtomicCpuId` to `AtomicCpuId::NONE`, i.e. makes /// Sets the inner value of an `AtomicCpuId` to `AtomicCpuId::NONE`, i.e. makes
@ -48,19 +53,19 @@ impl AtomicCpuId {
} }
/// Gets the inner value of an `AtomicCpuId`. /// Gets the inner value of an `AtomicCpuId`.
pub fn get(&self) -> Option<u32> { pub fn get(&self) -> Option<CpuId> {
let val = self.0.load(Ordering::Relaxed); let val = self.0.load(Ordering::Relaxed);
if val == Self::NONE { if val == Self::NONE {
None None
} else { } else {
Some(val) Some((val as usize).try_into().ok()?)
} }
} }
} }
impl Default for AtomicCpuId { impl Default for AtomicCpuId {
fn default() -> Self { fn default() -> Self {
Self::new(Self::NONE) Self(AtomicU32::new(Self::NONE))
} }
} }

View File

@ -13,7 +13,12 @@ use core::sync::atomic::{AtomicBool, Ordering};
use spin::Once; use spin::Once;
use super::{preempt::cpu_local, processor, Task}; use super::{preempt::cpu_local, processor, Task};
use crate::{cpu::PinCurrentCpu, prelude::*, task::disable_preempt, timer}; use crate::{
cpu::{CpuId, PinCurrentCpu},
prelude::*,
task::disable_preempt,
timer,
};
/// Injects a scheduler implementation into framework. /// Injects a scheduler implementation into framework.
/// ///
@ -40,7 +45,7 @@ pub trait Scheduler<T = Task>: Sync + Send {
/// ///
/// If the `current` of a CPU needs to be preempted, this method returns the id of /// If the `current` of a CPU needs to be preempted, this method returns the id of
/// that CPU. /// that CPU.
fn enqueue(&self, runnable: Arc<T>, flags: EnqueueFlags) -> Option<u32>; fn enqueue(&self, runnable: Arc<T>, flags: EnqueueFlags) -> Option<CpuId>;
/// Gets an immutable access to the local runqueue of the current CPU core. /// Gets an immutable access to the local runqueue of the current CPU core.
fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<T>)); fn local_rq_with(&self, f: &mut dyn FnMut(&dyn LocalRunQueue<T>));