/* * litmus/sched_global_plugin.c * * Implementation of the basic operations and architecture needed by * G-EDF/G-FIFO/EDZL/AEDZL global schedulers. * * This version uses the simple approach and serializes all scheduling * decisions by the use of a queue lock. This is probably not the * best way to do it, but it should suffice for now. */ #include #include #include #include #include #include #include #include #include /* Overview of Global operations. * * gbl_link_task_to_cpu(T, cpu) - Low-level operation to update the linkage * structure (NOT the actually scheduled * task). If there is another linked task To * already it will set To->linked_on = NO_CPU * (thereby removing its association with this * CPU). However, it will not requeue the * previously linked task (if any). It will set * T's state to RT_F_RUNNING and check whether * it is already running somewhere else. If T * is scheduled somewhere else it will link * it to that CPU instead (and pull the linked * task to cpu). T may be NULL. * * gbl_unlink(T) - Unlink removes T from all scheduler data * structures. If it is linked to some CPU it * will link NULL to that CPU. If it is * currently queued in the gsnedf queue it will * be removed from the rt_domain. It is safe to * call gbl_unlink(T) if T is not linked. T may not * be NULL. * * gbl_requeue(T) - Requeue will insert T into the appropriate * queue. If the system is in real-time mode and * the T is released already, it will go into the * ready queue. If the system is not in * real-time mode is T, then T will go into the * release queue. If T's release time is in the * future, it will go into the release * queue. That means that T's release time/job * no/etc. has to be updated before requeu(T) is * called. It is not safe to call gbl_requeue(T) * when T is already queued. T may not be NULL. * * job_arrival(T) - This is the catch all function when T enters * the system after either a suspension or at a * job release. It will queue T (which means it * is not safe to call job_arrival(T) if * T is already queued) and then check whether a * preemption is necessary. If a preemption is * necessary it will update the linkage * accordingly and cause scheduled to be called * (either with an IPI or need_resched). It is * safe to call job_arrival(T) if T's * next job has not been actually released yet * (releast time in the future). T will be put * on the release queue in that case. * * job_completion(T) - Take care of everything that needs to be done * to prepare T for its next release and place * it in the right queue with * job_arrival(). * * * When we now that T is linked to CPU then gbl_link_task_to_cpu(NULL, CPU) is * equivalent to gbl_unlink(T). Note that if you unlink a task from a CPU none of * the functions will automatically propagate pending task from the ready queue * to a linked task. This is the job of the calling function (by means of * __take_ready). */ /* Uncomment this if you want to see all scheduling decisions in the * TRACE() log. #define WANT_ALL_SCHED_EVENTS */ /* Macros to access the current active global plugin. These are * a lot like C++'s 'this' pointer. */ struct sched_global_plugin* active_gbl_plugin; #define active_gbl_domain (active_gbl_plugin->domain) #define active_gbl_domain_lock (active_gbl_domain.ready_lock) /*********************************************************************/ /* "Member" functions for both sched_plugin and sched_global_plugin. */ /* NOTE: These will automatically call down into "virtual" functions.*/ /*********************************************************************/ /* Priority-related functions */ int gbl_ready_order(struct bheap_node* a, struct bheap_node* b) { return active_gbl_plugin->prio_order(bheap2task(a), bheap2task(b)); } int gbl_cpu_lower_prio(struct bheap_node *_a, struct bheap_node *_b) { cpu_entry_t *a, *b; a = _a->value; b = _b->value; /* Note that a and b are inverted: we want the lowest-priority CPU at * the top of the heap. */ return active_gbl_plugin->prio_order(b->linked, a->linked); } /* gbl_update_cpu_position - Move the cpu entry to the correct place to maintain * order in the cpu queue. Caller must hold gbl_domain_lock. */ void gbl_update_cpu_position(cpu_entry_t *entry) { if (likely(bheap_node_in_heap(entry->hn))) bheap_delete(gbl_cpu_lower_prio, &active_gbl_plugin->cpu_heap, entry->hn); bheap_insert(gbl_cpu_lower_prio, &active_gbl_plugin->cpu_heap, entry->hn); } /* caller must hold gsnedf lock */ cpu_entry_t* lowest_prio_cpu(void) { struct bheap_node* hn; hn = bheap_peek(gbl_cpu_lower_prio, &active_gbl_plugin->cpu_heap); return hn->value; } /* link_task_to_cpu - Update the link of a CPU. * Handles the case where the to-be-linked task is already * scheduled on a different CPU. */ void gbl_link_task_to_cpu(struct task_struct* linked, cpu_entry_t *entry) { cpu_entry_t *sched; struct task_struct* tmp; int on_cpu; BUG_ON(linked && !is_realtime(linked)); /* Currently linked task is set to be unlinked. */ if (entry->linked) { entry->linked->rt_param.linked_on = NO_CPU; } /* Link new task to CPU. */ if (linked) { set_rt_flags(linked, RT_F_RUNNING); /* handle task is already scheduled somewhere! */ on_cpu = linked->rt_param.scheduled_on; if (on_cpu != NO_CPU) { sched = active_gbl_plugin->cpus[on_cpu]; /* this should only happen if not linked already */ BUG_ON(sched->linked == linked); /* If we are already scheduled on the CPU to which we * wanted to link, we don't need to do the swap -- * we just link ourselves to the CPU and depend on * the caller to get things right. */ if (entry != sched) { TRACE_TASK(linked, "already scheduled on %d, updating link.\n", sched->cpu); tmp = sched->linked; linked->rt_param.linked_on = sched->cpu; sched->linked = linked; gbl_update_cpu_position(sched); linked = tmp; } } if (linked) /* might be NULL due to swap */ linked->rt_param.linked_on = entry->cpu; } entry->linked = linked; #ifdef WANT_ALL_SCHED_EVENTS if (linked) TRACE_TASK(linked, "linked to %d.\n", entry->cpu); else TRACE("NULL linked to %d.\n", entry->cpu); #endif gbl_update_cpu_position(entry); } /* unlink - Make sure a task is not linked any longer to an entry * where it was linked before. Must hold * active_gbl_domain_lock. */ void gbl_unlink(struct task_struct* t) { cpu_entry_t *entry; if (t->rt_param.linked_on != NO_CPU) { /* unlink */ entry = active_gbl_plugin->cpus[t->rt_param.linked_on]; t->rt_param.linked_on = NO_CPU; gbl_link_task_to_cpu(NULL, entry); } else if (is_queued(t)) { /* This is an interesting situation: t is scheduled, * but was just recently unlinked. It cannot be * linked anywhere else (because then it would have * been relinked to this CPU), thus it must be in some * queue. We must remove it from the list in this * case. */ remove(&active_gbl_domain, t); } } /* preempt - force a CPU to reschedule */ void gbl_preempt(cpu_entry_t *entry) { preempt_if_preemptable(entry->scheduled, entry->cpu); } /* requeue - Put an unlinked task into global domain. * Caller must hold active_gbl_domain. */ void gbl_requeue(struct task_struct* task) { BUG_ON(!task); /* sanity check before insertion */ BUG_ON(is_queued(task)); if (is_released(task, litmus_clock())) active_gbl_plugin->add_ready(&active_gbl_domain, task); else { /* it has got to wait */ add_release(&active_gbl_domain, task); } } /* * update_queue_position - call after changing the priority of 'task'. */ void gbl_update_queue_position(struct task_struct *task) { /* We don't know whether task is in the ready queue. It should, but * on a budget overrun it may already be in a release queue. Hence, * calling unlink() is not possible since it assumes that the task is * not in a release queue. */ /* Assumption: caller holds active_gbl_domain_lock */ int check_preempt = 0; if (tsk_rt(task)->linked_on != NO_CPU) { TRACE_TASK(task, "%s: linked on %d\n", __FUNCTION__, tsk_rt(task)->linked_on); /* Task is scheduled; need to re-order CPUs. * We can't use heap_decrease() here since * the cpu_heap is ordered in reverse direction, so * it is actually an increase. */ bheap_delete(gbl_cpu_lower_prio, &active_gbl_plugin->cpu_heap, active_gbl_plugin->cpus[tsk_rt(task)->linked_on]->hn); bheap_insert(gbl_cpu_lower_prio, &active_gbl_plugin->cpu_heap, active_gbl_plugin->cpus[tsk_rt(task)->linked_on]->hn); } else { /* task may be queued: first stop queue changes */ raw_spin_lock(&active_gbl_domain.release_lock); if (is_queued(task)) { TRACE_TASK(task, "%s: is queued\n", __FUNCTION__); /* We need to update the position * of task in some heap. Note that this * may be a release heap. */ check_preempt = !bheap_decrease(gbl_ready_order, tsk_rt(task)->heap_node); } else { /* Nothing to do: if it is not queued and not linked * then it is currently being moved by other code * (e.g., a timer interrupt handler) that will use the * correct priority when enqueuing the task. */ TRACE_TASK(task, "%s: is NOT queued => Done.\n", __FUNCTION__); } raw_spin_unlock(&active_gbl_domain.release_lock); /* If task was enqueued in a release heap, then the following * preemption check is pointless, but we can't easily detect * that case. If you want to fix this, then consider that * simply adding a state flag requires O(n) time to update when * releasing n tasks, which conflicts with the goal to have * O(log n) merges. */ if (check_preempt) { /* heap_decrease() hit the top level of the heap: make * sure preemption checks get the right task, not the * potentially stale cache. */ bheap_uncache_min(gbl_ready_order, &active_gbl_domain.ready_queue); gbl_check_for_preemptions(); } } } /* check for any necessary preemptions */ void gbl_check_for_preemptions(void) { struct task_struct *task; cpu_entry_t* last; for(last = lowest_prio_cpu(); active_gbl_plugin->preemption_needed(last->linked); last = lowest_prio_cpu()) { /* preemption necessary */ task = active_gbl_plugin->take_ready(&active_gbl_domain); TRACE("check_for_preemptions: attempting to link task %d to %d\n", task->pid, last->cpu); if (last->linked) gbl_requeue(last->linked); gbl_link_task_to_cpu(task, last); gbl_preempt(last); } } void gbl_release_jobs(rt_domain_t* rt, struct bheap* tasks) { unsigned long flags; raw_spin_lock_irqsave(&active_gbl_domain_lock, flags); __merge_ready(rt, tasks); gbl_check_for_preemptions(); raw_spin_unlock_irqrestore(&active_gbl_domain_lock, flags); } /* caller holds active_gbl_domain_lock */ void gbl_job_completion(struct task_struct *t, int forced) { BUG_ON(!t); sched_trace_task_completion(t, forced); TRACE_TASK(t, "job_completion().\n"); /* set flags */ set_rt_flags(t, RT_F_SLEEP); /* prepare for next period */ prepare_for_next_period(t); if (is_released(t, litmus_clock())) sched_trace_task_release(t); /* unlink */ gbl_unlink(t); /* requeue * But don't requeue a blocking task. */ if (is_running(t)) active_gbl_plugin->job_arrival(t); } /*********************************************************************/ /* These two functions can't use active_* defines since the 'litmus' */ /* pointer is undefined/invalid when these are called. Think of them */ /* as static member functions. */ /*********************************************************************/ void gbl_domain_init(struct sched_global_plugin* gbl_plugin, check_resched_needed_t resched, release_jobs_t release) { rt_domain_init(&gbl_plugin->domain, gbl_ready_order, resched, release); } long gbl_activate_plugin(void* plg) { struct sched_plugin* plugin = (struct sched_plugin*)plg; int cpu; cpu_entry_t *entry; /* set the active global plugin */ active_gbl_plugin = container_of(plugin, struct sched_global_plugin, plugin); bheap_init(&active_gbl_plugin->cpu_heap); #ifdef CONFIG_RELEASE_MASTER active_gbl_domain.release_master = atomic_read(&release_master_cpu); #endif for_each_online_cpu(cpu) { entry = active_gbl_plugin->cpus[cpu]; bheap_node_init(&entry->hn, entry); entry->linked = NULL; entry->scheduled = NULL; #ifdef CONFIG_RELEASE_MASTER if (cpu != active_gbl_domain.release_master) { #endif TRACE("Global Plugin: Initializing CPU #%d.\n", cpu); gbl_update_cpu_position(entry); #ifdef CONFIG_RELEASE_MASTER } else { TRACE("Global Plugin: CPU %d is release master.\n", cpu); } #endif } return 0; } /********************************************************************/ /* "Virtual" functions in both sched_plugin and sched_global_plugin */ /********************************************************************/ /* gbl_job_arrival: task is either resumed or released */ void gblv_job_arrival(struct task_struct* task) { BUG_ON(!task); gbl_requeue(task); gbl_check_for_preemptions(); } int gblv_preemption_needed(struct task_struct *t) { /* we need the read lock for active_gbl_domain's ready_queue */ /* no need to preempt if there is nothing pending */ if (!__jobs_pending(&active_gbl_domain)) return 0; /* we need to reschedule if t doesn't exist */ if (!t) return 1; /* NOTE: We cannot check for non-preemptibility since we * don't know what address space we're currently in. */ /* make sure to get non-rt stuff out of the way */ return !is_realtime(t) || active_gbl_plugin->prio_order(__next_ready(&active_gbl_domain), t); } /* gbl_tick - this function is called for every local timer interrupt. * * checks whether the current task has expired and checks * whether we need to preempt it if it has not expired */ void gblv_tick(struct task_struct* t) { if (is_realtime(t) && budget_enforced(t) && budget_exhausted(t)) { if (!is_np(t)) { /* np tasks will be preempted when they become * preemptable again */ litmus_reschedule_local(); TRACE("gbl_scheduler_tick: " "%d is preemptable " " => FORCE_RESCHED\n", t->pid); } else if (is_user_np(t)) { TRACE("gbl_scheduler_tick: " "%d is non-preemptable, " "preemption delayed.\n", t->pid); request_exit_np(t); } } } /* Getting schedule() right is a bit tricky. schedule() may not make any * assumptions on the state of the current task since it may be called for a * number of reasons. The reasons include a scheduler_tick() determined that it * was necessary, because sys_exit_np() was called, because some Linux * subsystem determined so, or even (in the worst case) because there is a bug * hidden somewhere. Thus, we must take extreme care to determine what the * current state is. * * The CPU could currently be scheduling a task (or not), be linked (or not). * * The following assertions for the scheduled task could hold: * * - !is_running(scheduled) // the job blocks * - scheduled->timeslice == 0 // the job completed (forcefully) * - get_rt_flag() == RT_F_SLEEP // the job completed (by syscall) * - linked != scheduled // we need to reschedule (for any reason) * - is_np(scheduled) // rescheduling must be delayed, * sys_exit_np must be requested * * Any of these can occur together. */ struct task_struct* gblv_schedule(struct task_struct * prev) { cpu_entry_t* entry = active_gbl_plugin->cpus[smp_processor_id()]; int out_of_time, sleep, preempt, np, exists, blocks; struct task_struct* next = NULL; #ifdef CONFIG_RELEASE_MASTER /* Bail out early if we are the release master. * The release master never schedules any real-time tasks. */ if (active_gbl_domain.release_master == entry->cpu) return NULL; #endif raw_spin_lock(&active_gbl_domain_lock); /* sanity checking */ BUG_ON(entry->scheduled && entry->scheduled != prev); BUG_ON(entry->scheduled && !is_realtime(prev)); BUG_ON(is_realtime(prev) && !entry->scheduled); /* (0) Determine state */ exists = entry->scheduled != NULL; blocks = exists && !is_running(entry->scheduled); out_of_time = exists && budget_enforced(entry->scheduled) && budget_exhausted(entry->scheduled); np = exists && is_np(entry->scheduled); sleep = exists && get_rt_flags(entry->scheduled) == RT_F_SLEEP; preempt = entry->scheduled != entry->linked; #ifdef WANT_ALL_SCHED_EVENTS TRACE_TASK(prev, "invoked gsnedf_schedule.\n"); #endif if (exists) TRACE_TASK(prev, "blocks:%d out_of_time:%d np:%d sleep:%d preempt:%d " "state:%d sig:%d\n", blocks, out_of_time, np, sleep, preempt, prev->state, signal_pending(prev)); if (entry->linked && preempt) TRACE_TASK(prev, "will be preempted by %s/%d\n", entry->linked->comm, entry->linked->pid); /* If a task blocks we have no choice but to reschedule. */ if (blocks) gbl_unlink(entry->scheduled); /* Request a sys_exit_np() call if we would like to preempt but cannot. * We need to make sure to update the link structure anyway in case * that we are still linked. Multiple calls to request_exit_np() don't * hurt. */ if (np && (out_of_time || preempt || sleep)) { gbl_unlink(entry->scheduled); request_exit_np(entry->scheduled); } /* Any task that is preemptable and either exhausts its execution * budget or wants to sleep completes. We may have to reschedule after * this. Don't do a job completion if we block (can't have timers running * for blocked jobs). Preemption go first for the same reason. */ if (!np && (out_of_time || sleep) && !blocks && !preempt) active_gbl_plugin->job_completion(entry->scheduled, !sleep); /* Link pending task if we became unlinked. */ if (!entry->linked) gbl_link_task_to_cpu(active_gbl_plugin->take_ready(&active_gbl_domain), entry); /* The final scheduling decision. Do we need to switch for some reason? * If linked is different from scheduled, then select linked as next. */ if ((!np || blocks) && entry->linked != entry->scheduled) { /* Schedule a linked job? */ if (entry->linked) { entry->linked->rt_param.scheduled_on = entry->cpu; next = entry->linked; } if (entry->scheduled) { /* not gonna be scheduled soon */ entry->scheduled->rt_param.scheduled_on = NO_CPU; TRACE_TASK(entry->scheduled, "scheduled_on = NO_CPU\n"); } } else /* Only override Linux scheduler if we have a real-time task * scheduled that needs to continue. */ if (exists) next = prev; sched_state_task_picked(); raw_spin_unlock(&active_gbl_domain_lock); #ifdef WANT_ALL_SCHED_EVENTS TRACE("active_gbl_domain_lock released, next=0x%p\n", next); if (next) TRACE_TASK(next, "scheduled at %llu\n", litmus_clock()); else if (exists && !next) TRACE("becomes idle at %llu.\n", litmus_clock()); #endif return next; } /* _finish_switch - we just finished the switch away from prev */ void gblv_finish_switch(struct task_struct *prev) { cpu_entry_t* entry = active_gbl_plugin->cpus[smp_processor_id()]; entry->scheduled = is_realtime(current) ? current : NULL; #ifdef WANT_ALL_SCHED_EVENTS TRACE_TASK(prev, "switched away from\n"); #endif } /* Prepare a task for running in RT mode */ void gblv_task_new(struct task_struct * t, int on_rq, int running) { unsigned long flags; cpu_entry_t* entry; TRACE("global plugin: task new %d\n", t->pid); raw_spin_lock_irqsave(&active_gbl_domain_lock, flags); /* setup job params */ release_at(t, litmus_clock()); if (running) { entry = active_gbl_plugin->cpus[task_cpu(t)]; BUG_ON(entry->scheduled); #ifdef CONFIG_RELEASE_MASTER if (entry->cpu != active_gbl_domain.release_master) { #endif entry->scheduled = t; tsk_rt(t)->scheduled_on = task_cpu(t); #ifdef CONFIG_RELEASE_MASTER } else { /* do not schedule on release master */ gbl_preempt(entry); /* force resched */ tsk_rt(t)->scheduled_on = NO_CPU; } #endif } else { t->rt_param.scheduled_on = NO_CPU; } t->rt_param.linked_on = NO_CPU; active_gbl_plugin->job_arrival(t); raw_spin_unlock_irqrestore(&active_gbl_domain_lock, flags); } void gblv_task_wake_up(struct task_struct *task) { unsigned long flags; lt_t now; TRACE_TASK(task, "wake_up at %llu\n", litmus_clock()); raw_spin_lock_irqsave(&active_gbl_domain_lock, flags); /* We need to take suspensions because of semaphores into * account! If a job resumes after being suspended due to acquiring * a semaphore, it should never be treated as a new job release. */ if (get_rt_flags(task) == RT_F_EXIT_SEM) { set_rt_flags(task, RT_F_RUNNING); } else { now = litmus_clock(); if (is_tardy(task, now)) { /* new sporadic release */ release_at(task, now); sched_trace_task_release(task); } else { if (task->rt.time_slice) { /* came back in time before deadline */ set_rt_flags(task, RT_F_RUNNING); } } } active_gbl_plugin->job_arrival(task); raw_spin_unlock_irqrestore(&active_gbl_domain_lock, flags); } void gblv_task_block(struct task_struct *t) { unsigned long flags; TRACE_TASK(t, "block at %llu\n", litmus_clock()); /* unlink if necessary */ raw_spin_lock_irqsave(&active_gbl_domain_lock, flags); gbl_unlink(t); raw_spin_unlock_irqrestore(&active_gbl_domain_lock, flags); BUG_ON(!is_realtime(t)); } void gblv_task_exit(struct task_struct * t) { unsigned long flags; /* unlink if necessary */ raw_spin_lock_irqsave(&active_gbl_domain_lock, flags); gbl_unlink(t); if (tsk_rt(t)->scheduled_on != NO_CPU) { active_gbl_plugin->cpus[tsk_rt(t)->scheduled_on]->scheduled = NULL; tsk_rt(t)->scheduled_on = NO_CPU; } raw_spin_unlock_irqrestore(&active_gbl_domain_lock, flags); BUG_ON(!is_realtime(t)); TRACE_TASK(t, "RIP\n"); } long gblv_admit_task(struct task_struct* tsk) { return 0; }